diff --git a/.golangci.yml b/.golangci.yml index a4a087333ed3c..816e2404a9e36 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -28,9 +28,6 @@ linters-settings: checks: ["S1002","S1004","S1007","S1009","S1010","S1012","S1019","S1020","S1021","S1024","S1030","SA2*","SA3*","SA4009","SA5*","SA6000","SA6001","SA6005", "-SA2002"] stylecheck: checks: ["-ST1003"] - gosec: - excludes: - - G601 issues: exclude-rules: - path: _test\.go diff --git a/bindinfo/bind_serial_test.go b/bindinfo/bind_serial_test.go index 41a88ec0e893d..49ca023a1213b 100644 --- a/bindinfo/bind_serial_test.go +++ b/bindinfo/bind_serial_test.go @@ -555,7 +555,7 @@ func TestHintsSetID(t *testing.T) { utilCleanBindingEnv(tk, dom) err := tk.ExecToErr("create global binding for select * from t using select /*+ non_exist_hint() */ * from t") - require.True(t, terror.ErrorEqual(err, parser.ErrWarnOptimizerHintParseError)) + require.True(t, terror.ErrorEqual(err, parser.ErrParse)) tk.MustExec("create global binding for select * from t where a > 10 using select * from t where a > 10") bindData = bindHandle.GetBindRecord(hash, sql, "test") require.NotNil(t, bindData) diff --git a/bindinfo/cache.go b/bindinfo/cache.go index 26a7830b4fe0e..9cdff845c9e6b 100644 --- a/bindinfo/cache.go +++ b/bindinfo/cache.go @@ -105,7 +105,8 @@ func (br *BindRecord) HasUsingBinding() bool { // FindBinding find bindings in BindRecord. func (br *BindRecord) FindBinding(hint string) *Binding { - for _, binding := range br.Bindings { + for i := range br.Bindings { + binding := br.Bindings[i] if binding.ID == hint { return &binding } @@ -160,7 +161,8 @@ func merge(lBindRecord, rBindRecord *BindRecord) *BindRecord { return lBindRecord } result := lBindRecord.shallowCopy() - for _, rbind := range rBindRecord.Bindings { + for i := range rBindRecord.Bindings { + rbind := rBindRecord.Bindings[i] found := false for j, lbind := range lBindRecord.Bindings { if lbind.isSame(&rbind) { @@ -184,7 +186,8 @@ func (br *BindRecord) remove(deleted *BindRecord) *BindRecord { return &BindRecord{OriginalSQL: br.OriginalSQL, Db: br.Db} } result := br.shallowCopy() - for _, deletedBind := range deleted.Bindings { + for j := range deleted.Bindings { + deletedBind := deleted.Bindings[j] for i, bind := range result.Bindings { if bind.isSame(&deletedBind) { result.Bindings = append(result.Bindings[:i], result.Bindings[i+1:]...) diff --git a/br/cmd/tidb-lightning-ctl/main.go b/br/cmd/tidb-lightning-ctl/main.go index b80da9dc7a212..43891c8fb0fb3 100644 --- a/br/cmd/tidb-lightning-ctl/main.go +++ b/br/cmd/tidb-lightning-ctl/main.go @@ -229,8 +229,8 @@ func checkpointErrorDestroy(ctx context.Context, cfg *config.Config, tls *common for engineID := table.MinEngineID; engineID <= table.MaxEngineID; engineID++ { fmt.Fprintln(os.Stderr, "Closing and cleaning up engine:", table.TableName, engineID) _, eID := backend.MakeUUID(table.TableName, engineID) - file := local.File{UUID: eID} - err := file.Cleanup(cfg.TikvImporter.SortedKVDir) + engine := local.Engine{UUID: eID} + err := engine.Cleanup(cfg.TikvImporter.SortedKVDir) if err != nil { fmt.Fprintln(os.Stderr, "* Encountered error while cleanup engine:", err) lastErr = err diff --git a/br/cmd/tidb-lightning/main.go b/br/cmd/tidb-lightning/main.go index 083e47e82d65d..84362433f222c 100644 --- a/br/cmd/tidb-lightning/main.go +++ b/br/cmd/tidb-lightning/main.go @@ -30,7 +30,10 @@ import ( func main() { globalCfg := config.Must(config.LoadGlobalConfig(os.Args[1:], nil)) - fmt.Fprintf(os.Stdout, "Verbose debug logs will be written to %s\n\n", globalCfg.App.Config.File) + logToFile := globalCfg.App.File != "" && globalCfg.App.File != "-" + if logToFile { + fmt.Fprintf(os.Stdout, "Verbose debug logs will be written to %s\n\n", globalCfg.App.Config.File) + } app := lightning.New(globalCfg) @@ -95,7 +98,7 @@ func main() { } // call Sync() with log to stdout may return error in some case, so just skip it - if globalCfg.App.File != "" { + if logToFile { syncErr := logger.Sync() if syncErr != nil { fmt.Fprintln(os.Stderr, "sync log failed", syncErr) diff --git a/br/pkg/lightning/backend/backend_test.go b/br/pkg/lightning/backend/backend_test.go index e2ca9703a8a16..80ee77dff9b7c 100644 --- a/br/pkg/lightning/backend/backend_test.go +++ b/br/pkg/lightning/backend/backend_test.go @@ -7,12 +7,12 @@ import ( "github.com/golang/mock/gomock" "github.com/google/uuid" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/lightning/backend" "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" "github.com/pingcap/tidb/br/pkg/mock" "github.com/pingcap/tidb/parser/mysql" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" ) @@ -23,30 +23,25 @@ type backendSuite struct { ts uint64 } -var _ = Suite(&backendSuite{}) - -func Test(t *testing.T) { - TestingT(t) -} - -// FIXME: Cannot use the real SetUpTest/TearDownTest to set up the mock -// otherwise the mock error will be ignored. - -func (s *backendSuite) setUpTest(c gomock.TestReporter) { - s.controller = gomock.NewController(c) - s.mockBackend = mock.NewMockBackend(s.controller) - s.backend = backend.MakeBackend(s.mockBackend) - s.ts = oracle.ComposeTS(time.Now().Unix()*1000, 0) +func createBackendSuite(c gomock.TestReporter) *backendSuite { + controller := gomock.NewController(c) + mockBackend := mock.NewMockBackend(controller) + return &backendSuite{ + controller: controller, + mockBackend: mockBackend, + backend: backend.MakeBackend(mockBackend), + ts: oracle.ComposeTS(time.Now().Unix()*1000, 0), + } } func (s *backendSuite) tearDownTest() { s.controller.Finish() } -func (s *backendSuite) TestOpenCloseImportCleanUpEngine(c *C) { - s.setUpTest(c) +func TestOpenCloseImportCleanUpEngine(t *testing.T) { + t.Parallel() + s := createBackendSuite(t) defer s.tearDownTest() - ctx := context.Background() engineUUID := uuid.MustParse("902efee3-a3f9-53d4-8c82-f12fb1900cd1") @@ -67,17 +62,18 @@ func (s *backendSuite) TestOpenCloseImportCleanUpEngine(c *C) { After(importCall) engine, err := s.backend.OpenEngine(ctx, &backend.EngineConfig{}, "`db`.`table`", 1) - c.Assert(err, IsNil) + require.NoError(t, err) closedEngine, err := engine.Close(ctx, nil) - c.Assert(err, IsNil) + require.NoError(t, err) err = closedEngine.Import(ctx, 1) - c.Assert(err, IsNil) + require.NoError(t, err) err = closedEngine.Cleanup(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *backendSuite) TestUnsafeCloseEngine(c *C) { - s.setUpTest(c) +func TestUnsafeCloseEngine(t *testing.T) { + t.Parallel() + s := createBackendSuite(t) defer s.tearDownTest() ctx := context.Background() @@ -92,13 +88,14 @@ func (s *backendSuite) TestUnsafeCloseEngine(c *C) { After(closeCall) closedEngine, err := s.backend.UnsafeCloseEngine(ctx, nil, "`db`.`table`", -1) - c.Assert(err, IsNil) + require.NoError(t, err) err = closedEngine.Cleanup(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *backendSuite) TestUnsafeCloseEngineWithUUID(c *C) { - s.setUpTest(c) +func TestUnsafeCloseEngineWithUUID(t *testing.T) { + t.Parallel() + s := createBackendSuite(t) defer s.tearDownTest() ctx := context.Background() @@ -113,13 +110,14 @@ func (s *backendSuite) TestUnsafeCloseEngineWithUUID(c *C) { After(closeCall) closedEngine, err := s.backend.UnsafeCloseEngineWithUUID(ctx, nil, "some_tag", engineUUID) - c.Assert(err, IsNil) + require.NoError(t, err) err = closedEngine.Cleanup(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *backendSuite) TestWriteEngine(c *C) { - s.setUpTest(c) +func TestWriteEngine(t *testing.T) { + t.Parallel() + s := createBackendSuite(t) defer s.tearDownTest() ctx := context.Background() @@ -145,19 +143,20 @@ func (s *backendSuite) TestWriteEngine(c *C) { Return(nil) engine, err := s.backend.OpenEngine(ctx, &backend.EngineConfig{}, "`db`.`table`", 1) - c.Assert(err, IsNil) + require.NoError(t, err) writer, err := engine.LocalWriter(ctx, &backend.LocalWriterConfig{}) - c.Assert(err, IsNil) + require.NoError(t, err) err = writer.WriteRows(ctx, []string{"c1", "c2"}, rows1) - c.Assert(err, IsNil) + require.NoError(t, err) err = writer.WriteRows(ctx, []string{"c1", "c2"}, rows2) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = writer.Close(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *backendSuite) TestWriteToEngineWithNothing(c *C) { - s.setUpTest(c) +func TestWriteToEngineWithNothing(t *testing.T) { + t.Parallel() + s := createBackendSuite(t) defer s.tearDownTest() ctx := context.Background() @@ -170,17 +169,18 @@ func (s *backendSuite) TestWriteToEngineWithNothing(c *C) { s.mockBackend.EXPECT().LocalWriter(ctx, &backend.LocalWriterConfig{}, gomock.Any()).Return(mockWriter, nil) engine, err := s.backend.OpenEngine(ctx, &backend.EngineConfig{}, "`db`.`table`", 1) - c.Assert(err, IsNil) + require.NoError(t, err) writer, err := engine.LocalWriter(ctx, &backend.LocalWriterConfig{}) - c.Assert(err, IsNil) + require.NoError(t, err) err = writer.WriteRows(ctx, nil, emptyRows) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = writer.Close(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *backendSuite) TestOpenEngineFailed(c *C) { - s.setUpTest(c) +func TestOpenEngineFailed(t *testing.T) { + t.Parallel() + s := createBackendSuite(t) defer s.tearDownTest() ctx := context.Background() @@ -189,11 +189,12 @@ func (s *backendSuite) TestOpenEngineFailed(c *C) { Return(errors.New("fake unrecoverable open error")) _, err := s.backend.OpenEngine(ctx, &backend.EngineConfig{}, "`db`.`table`", 1) - c.Assert(err, ErrorMatches, "fake unrecoverable open error") + require.EqualError(t, err, "fake unrecoverable open error") } -func (s *backendSuite) TestWriteEngineFailed(c *C) { - s.setUpTest(c) +func TestWriteEngineFailed(t *testing.T) { + t.Parallel() + s := createBackendSuite(t) defer s.tearDownTest() ctx := context.Background() @@ -209,17 +210,19 @@ func (s *backendSuite) TestWriteEngineFailed(c *C) { mockWriter.EXPECT().Close(ctx).Return(nil, nil) engine, err := s.backend.OpenEngine(ctx, &backend.EngineConfig{}, "`db`.`table`", 1) - c.Assert(err, IsNil) + require.NoError(t, err) writer, err := engine.LocalWriter(ctx, &backend.LocalWriterConfig{}) - c.Assert(err, IsNil) + require.NoError(t, err) err = writer.WriteRows(ctx, nil, rows) - c.Assert(err, ErrorMatches, "fake unrecoverable write error.*") + require.Error(t, err) + require.Regexp(t, "fake unrecoverable write error.*", err.Error()) _, err = writer.Close(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *backendSuite) TestWriteBatchSendFailedWithRetry(c *C) { - s.setUpTest(c) +func TestWriteBatchSendFailedWithRetry(t *testing.T) { + t.Parallel() + s := createBackendSuite(t) defer s.tearDownTest() ctx := context.Background() @@ -235,17 +238,19 @@ func (s *backendSuite) TestWriteBatchSendFailedWithRetry(c *C) { mockWriter.EXPECT().Close(ctx).Return(nil, nil).MinTimes(1) engine, err := s.backend.OpenEngine(ctx, &backend.EngineConfig{}, "`db`.`table`", 1) - c.Assert(err, IsNil) + require.NoError(t, err) writer, err := engine.LocalWriter(ctx, &backend.LocalWriterConfig{}) - c.Assert(err, IsNil) + require.NoError(t, err) err = writer.WriteRows(ctx, nil, rows) - c.Assert(err, ErrorMatches, ".*fake recoverable write batch error") + require.Error(t, err) + require.Regexp(t, ".*fake recoverable write batch error", err.Error()) _, err = writer.Close(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *backendSuite) TestImportFailedNoRetry(c *C) { - s.setUpTest(c) +func TestImportFailedNoRetry(t *testing.T) { + t.Parallel() + s := createBackendSuite(t) defer s.tearDownTest() ctx := context.Background() @@ -256,13 +261,15 @@ func (s *backendSuite) TestImportFailedNoRetry(c *C) { Return(errors.Annotate(context.Canceled, "fake unrecoverable import error")) closedEngine, err := s.backend.UnsafeCloseEngine(ctx, nil, "`db`.`table`", 1) - c.Assert(err, IsNil) + require.NoError(t, err) err = closedEngine.Import(ctx, 1) - c.Assert(err, ErrorMatches, "fake unrecoverable import error.*") + require.Error(t, err) + require.Regexp(t, "fake unrecoverable import error.*", err.Error()) } -func (s *backendSuite) TestImportFailedWithRetry(c *C) { - s.setUpTest(c) +func TestImportFailedWithRetry(t *testing.T) { + t.Parallel() + s := createBackendSuite(t) defer s.tearDownTest() ctx := context.Background() @@ -275,13 +282,15 @@ func (s *backendSuite) TestImportFailedWithRetry(c *C) { s.mockBackend.EXPECT().RetryImportDelay().Return(time.Duration(0)).AnyTimes() closedEngine, err := s.backend.UnsafeCloseEngine(ctx, nil, "`db`.`table`", 1) - c.Assert(err, IsNil) + require.NoError(t, err) err = closedEngine.Import(ctx, 1) - c.Assert(err, ErrorMatches, ".*fake recoverable import error") + require.Error(t, err) + require.Regexp(t, ".*fake recoverable import error", err.Error()) } -func (s *backendSuite) TestImportFailedRecovered(c *C) { - s.setUpTest(c) +func TestImportFailedRecovered(t *testing.T) { + t.Parallel() + s := createBackendSuite(t) defer s.tearDownTest() ctx := context.Background() @@ -296,14 +305,15 @@ func (s *backendSuite) TestImportFailedRecovered(c *C) { s.mockBackend.EXPECT().RetryImportDelay().Return(time.Duration(0)).AnyTimes() closedEngine, err := s.backend.UnsafeCloseEngine(ctx, nil, "`db`.`table`", 1) - c.Assert(err, IsNil) + require.NoError(t, err) err = closedEngine.Import(ctx, 1) - c.Assert(err, IsNil) + require.NoError(t, err) } //nolint:interfacer // change test case signature causes check panicking. -func (s *backendSuite) TestClose(c *C) { - s.setUpTest(c) +func TestClose(t *testing.T) { + t.Parallel() + s := createBackendSuite(t) defer s.tearDownTest() s.mockBackend.EXPECT().Close().Return() @@ -311,18 +321,19 @@ func (s *backendSuite) TestClose(c *C) { s.backend.Close() } -func (s *backendSuite) TestMakeEmptyRows(c *C) { - s.setUpTest(c) +func TestMakeEmptyRows(t *testing.T) { + t.Parallel() + s := createBackendSuite(t) defer s.tearDownTest() rows := mock.NewMockRows(s.controller) s.mockBackend.EXPECT().MakeEmptyRows().Return(rows) - - c.Assert(s.mockBackend.MakeEmptyRows(), Equals, rows) + require.Equal(t, rows, s.mockBackend.MakeEmptyRows()) } -func (s *backendSuite) TestNewEncoder(c *C) { - s.setUpTest(c) +func TestNewEncoder(t *testing.T) { + t.Parallel() + s := createBackendSuite(t) defer s.tearDownTest() encoder := mock.NewMockEncoder(s.controller) @@ -330,12 +341,13 @@ func (s *backendSuite) TestNewEncoder(c *C) { s.mockBackend.EXPECT().NewEncoder(nil, options).Return(encoder, nil) realEncoder, err := s.mockBackend.NewEncoder(nil, options) - c.Assert(realEncoder, Equals, encoder) - c.Assert(err, IsNil) + require.Equal(t, realEncoder, encoder) + require.NoError(t, err) } -func (s *backendSuite) TestCheckDiskQuota(c *C) { - s.setUpTest(c) +func TestCheckDiskQuota(t *testing.T) { + t.Parallel() + s := createBackendSuite(t) defer s.tearDownTest() uuid1 := uuid.MustParse("11111111-1111-1111-1111-111111111111") @@ -381,29 +393,29 @@ func (s *backendSuite) TestCheckDiskQuota(c *C) { // No quota exceeded le, iple, ds, ms := s.backend.CheckDiskQuota(30000) - c.Assert(le, HasLen, 0) - c.Assert(iple, Equals, 0) - c.Assert(ds, Equals, int64(9000)) - c.Assert(ms, Equals, int64(16000)) + require.Len(t, le, 0) + require.Equal(t, 0, iple) + require.Equal(t, int64(9000), ds) + require.Equal(t, int64(16000), ms) // Quota exceeded, the largest one is out le, iple, ds, ms = s.backend.CheckDiskQuota(20000) - c.Assert(le, DeepEquals, []uuid.UUID{uuid9}) - c.Assert(iple, Equals, 0) - c.Assert(ds, Equals, int64(9000)) - c.Assert(ms, Equals, int64(16000)) + require.Equal(t, []uuid.UUID{uuid9}, le) + require.Equal(t, 0, iple) + require.Equal(t, int64(9000), ds) + require.Equal(t, int64(16000), ms) // Quota exceeded, the importing one should be ranked least priority le, iple, ds, ms = s.backend.CheckDiskQuota(12000) - c.Assert(le, DeepEquals, []uuid.UUID{uuid5, uuid9}) - c.Assert(iple, Equals, 0) - c.Assert(ds, Equals, int64(9000)) - c.Assert(ms, Equals, int64(16000)) + require.Equal(t, []uuid.UUID{uuid5, uuid9}, le) + require.Equal(t, 0, iple) + require.Equal(t, int64(9000), ds) + require.Equal(t, int64(16000), ms) // Quota exceeded, the importing ones should not be visible le, iple, ds, ms = s.backend.CheckDiskQuota(5000) - c.Assert(le, DeepEquals, []uuid.UUID{uuid1, uuid5, uuid9}) - c.Assert(iple, Equals, 1) - c.Assert(ds, Equals, int64(9000)) - c.Assert(ms, Equals, int64(16000)) + require.Equal(t, []uuid.UUID{uuid1, uuid5, uuid9}, le) + require.Equal(t, 1, iple) + require.Equal(t, int64(9000), ds) + require.Equal(t, int64(16000), ms) } diff --git a/br/pkg/lightning/backend/importer/importer_test.go b/br/pkg/lightning/backend/importer/importer_test.go index 5d75d1badc245..a0d7878ea5ecb 100644 --- a/br/pkg/lightning/backend/importer/importer_test.go +++ b/br/pkg/lightning/backend/importer/importer_test.go @@ -24,13 +24,13 @@ import ( "github.com/golang/mock/gomock" "github.com/google/uuid" - . "github.com/pingcap/check" "github.com/pingcap/errors" kvpb "github.com/pingcap/kvproto/pkg/import_kvpb" "github.com/pingcap/tidb/br/pkg/lightning/backend" "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/mock" + "github.com/stretchr/testify/require" ) type importerSuite struct { @@ -43,20 +43,14 @@ type importerSuite struct { kvPairs kv.Rows } -var _ = Suite(&importerSuite{}) - const testPDAddr = "pd-addr:2379" -// FIXME: Cannot use the real SetUpTest/TearDownTest to set up the mock -// otherwise the mock error will be ignored. - -func (s *importerSuite) setUpTest(c *C) { - s.controller = gomock.NewController(c) - s.mockClient = mock.NewMockImportKVClient(s.controller) - s.mockWriter = mock.NewMockImportKV_WriteEngineClient(s.controller) - importer := NewMockImporter(s.mockClient, testPDAddr) - - s.ctx = context.Background() +func createImportSuite(t *testing.T) *importerSuite { + controller := gomock.NewController(t) + mockClient := mock.NewMockImportKVClient(controller) + mockWriter := mock.NewMockImportKV_WriteEngineClient(controller) + importer := NewMockImporter(mockClient, testPDAddr) + s := &importerSuite{controller: controller, mockClient: mockClient, mockWriter: mockWriter, ctx: context.Background()} engineUUID := uuid.MustParse("7e3f3a3c-67ce-506d-af34-417ec138fbcb") s.engineUUID = engineUUID[:] s.kvPairs = kv.MakeRowsFromKvPairs([]common.KvPair{ @@ -76,15 +70,17 @@ func (s *importerSuite) setUpTest(c *C) { var err error s.engine, err = importer.OpenEngine(s.ctx, &backend.EngineConfig{}, "`db`.`table`", -1) - c.Assert(err, IsNil) + require.NoError(t, err) + return s } func (s *importerSuite) tearDownTest() { s.controller.Finish() } -func (s *importerSuite) TestWriteRows(c *C) { - s.setUpTest(c) +func TestWriteRows(t *testing.T) { + t.Parallel() + s := createImportSuite(t) defer s.tearDownTest() s.mockClient.EXPECT().WriteEngine(s.ctx).Return(s.mockWriter, nil) @@ -99,10 +95,10 @@ func (s *importerSuite) TestWriteRows(c *C) { batchSendCall := s.mockWriter.EXPECT(). Send(gomock.Any()). DoAndReturn(func(x *kvpb.WriteEngineRequest) error { - c.Assert(x.GetBatch().GetMutations(), DeepEquals, []*kvpb.Mutation{ + require.Equal(t, []*kvpb.Mutation{ {Op: kvpb.Mutation_Put, Key: []byte("k1"), Value: []byte("v1")}, {Op: kvpb.Mutation_Put, Key: []byte("k2"), Value: []byte("v2")}, - }) + }, x.GetBatch().GetMutations()) return nil }). After(headSendCall) @@ -112,16 +108,17 @@ func (s *importerSuite) TestWriteRows(c *C) { After(batchSendCall) writer, err := s.engine.LocalWriter(s.ctx, nil) - c.Assert(err, IsNil) + require.NoError(t, err) err = writer.WriteRows(s.ctx, nil, s.kvPairs) - c.Assert(err, IsNil) + require.NoError(t, err) st, err := writer.Close(s.ctx) - c.Assert(err, IsNil) - c.Assert(st, IsNil) + require.NoError(t, err) + require.Nil(t, st) } -func (s *importerSuite) TestWriteHeadSendFailed(c *C) { - s.setUpTest(c) +func TestWriteHeadSendFailed(t *testing.T) { + t.Parallel() + s := createImportSuite(t) defer s.tearDownTest() s.mockClient.EXPECT().WriteEngine(s.ctx).Return(s.mockWriter, nil) @@ -129,7 +126,7 @@ func (s *importerSuite) TestWriteHeadSendFailed(c *C) { headSendCall := s.mockWriter.EXPECT(). Send(gomock.Any()). DoAndReturn(func(x *kvpb.WriteEngineRequest) error { - c.Assert(x.GetHead(), NotNil) + require.NotNil(t, x.GetHead()) return errors.Annotate(context.Canceled, "fake unrecoverable write head error") }) s.mockWriter.EXPECT(). @@ -138,13 +135,15 @@ func (s *importerSuite) TestWriteHeadSendFailed(c *C) { After(headSendCall) writer, err := s.engine.LocalWriter(s.ctx, nil) - c.Assert(err, IsNil) + require.NoError(t, err) err = writer.WriteRows(s.ctx, nil, s.kvPairs) - c.Assert(err, ErrorMatches, "fake unrecoverable write head error.*") + require.Error(t, err) + require.Regexp(t, "^fake unrecoverable write head error", err.Error()) } -func (s *importerSuite) TestWriteBatchSendFailed(c *C) { - s.setUpTest(c) +func TestWriteBatchSendFailed(t *testing.T) { + t.Parallel() + s := createImportSuite(t) defer s.tearDownTest() s.mockClient.EXPECT().WriteEngine(s.ctx).Return(s.mockWriter, nil) @@ -152,13 +151,13 @@ func (s *importerSuite) TestWriteBatchSendFailed(c *C) { headSendCall := s.mockWriter.EXPECT(). Send(gomock.Any()). DoAndReturn(func(x *kvpb.WriteEngineRequest) error { - c.Assert(x.GetHead(), NotNil) + require.NotNil(t, x.GetHead()) return nil }) batchSendCall := s.mockWriter.EXPECT(). Send(gomock.Any()). DoAndReturn(func(x *kvpb.WriteEngineRequest) error { - c.Assert(x.GetBatch(), NotNil) + require.NotNil(t, x.GetBatch()) return errors.Annotate(context.Canceled, "fake unrecoverable write batch error") }). After(headSendCall) @@ -168,13 +167,15 @@ func (s *importerSuite) TestWriteBatchSendFailed(c *C) { After(batchSendCall) writer, err := s.engine.LocalWriter(s.ctx, nil) - c.Assert(err, IsNil) + require.NoError(t, err) err = writer.WriteRows(s.ctx, nil, s.kvPairs) - c.Assert(err, ErrorMatches, "fake unrecoverable write batch error.*") + require.Error(t, err) + require.Regexp(t, "^fake unrecoverable write batch error", err.Error()) } -func (s *importerSuite) TestWriteCloseFailed(c *C) { - s.setUpTest(c) +func TestWriteCloseFailed(t *testing.T) { + t.Parallel() + s := createImportSuite(t) defer s.tearDownTest() s.mockClient.EXPECT().WriteEngine(s.ctx).Return(s.mockWriter, nil) @@ -182,13 +183,13 @@ func (s *importerSuite) TestWriteCloseFailed(c *C) { headSendCall := s.mockWriter.EXPECT(). Send(gomock.Any()). DoAndReturn(func(x *kvpb.WriteEngineRequest) error { - c.Assert(x.GetHead(), NotNil) + require.NotNil(t, x.GetHead()) return nil }) batchSendCall := s.mockWriter.EXPECT(). Send(gomock.Any()). DoAndReturn(func(x *kvpb.WriteEngineRequest) error { - c.Assert(x.GetBatch(), NotNil) + require.NotNil(t, x.GetBatch()) return nil }). After(headSendCall) @@ -198,13 +199,15 @@ func (s *importerSuite) TestWriteCloseFailed(c *C) { After(batchSendCall) writer, err := s.engine.LocalWriter(s.ctx, nil) - c.Assert(err, IsNil) + require.NoError(t, err) err = writer.WriteRows(s.ctx, nil, s.kvPairs) - c.Assert(err, ErrorMatches, "fake unrecoverable close stream error.*") + require.Error(t, err) + require.Regexp(t, "^fake unrecoverable close stream error", err.Error()) } -func (s *importerSuite) TestCloseImportCleanupEngine(c *C) { - s.setUpTest(c) +func TestCloseImportCleanupEngine(t *testing.T) { + t.Parallel() + s := createImportSuite(t) defer s.tearDownTest() s.mockClient.EXPECT(). @@ -218,11 +221,11 @@ func (s *importerSuite) TestCloseImportCleanupEngine(c *C) { Return(nil, nil) engine, err := s.engine.Close(s.ctx, nil) - c.Assert(err, IsNil) + require.NoError(t, err) err = engine.Import(s.ctx, 1) - c.Assert(err, IsNil) + require.NoError(t, err) err = engine.Cleanup(s.ctx) - c.Assert(err, IsNil) + require.NoError(t, err) } func BenchmarkMutationAlloc(b *testing.B) { @@ -261,33 +264,41 @@ func BenchmarkMutationPool(b *testing.B) { _ = g } -func (s *importerSuite) TestCheckTiDBVersion(c *C) { +func TestCheckTiDBVersion(t *testing.T) { var version string ctx := context.Background() mockServer := httptest.NewTLSServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { - c.Assert(req.URL.Path, Equals, "/status") + require.Equal(t, "/status", req.URL.Path) w.WriteHeader(http.StatusOK) err := json.NewEncoder(w).Encode(map[string]interface{}{ "version": version, }) - c.Assert(err, IsNil) + require.NoError(t, err) })) tls := common.NewTLSFromMockServer(mockServer) version = "5.7.25-TiDB-v4.0.0" - c.Assert(checkTiDBVersionByTLS(ctx, tls, requiredMinTiDBVersion, requiredMaxTiDBVersion), IsNil) + require.Nil(t, checkTiDBVersionByTLS(ctx, tls, requiredMinTiDBVersion, requiredMaxTiDBVersion)) version = "5.7.25-TiDB-v9999.0.0" - c.Assert(checkTiDBVersionByTLS(ctx, tls, requiredMinTiDBVersion, requiredMaxTiDBVersion), ErrorMatches, "TiDB version too new.*") + err := checkTiDBVersionByTLS(ctx, tls, requiredMinTiDBVersion, requiredMaxTiDBVersion) + require.Error(t, err) + require.Regexp(t, "^TiDB version too new", err.Error()) version = "5.7.25-TiDB-v6.0.0" - c.Assert(checkTiDBVersionByTLS(ctx, tls, requiredMinTiDBVersion, requiredMaxTiDBVersion), ErrorMatches, "TiDB version too new.*") + err = checkTiDBVersionByTLS(ctx, tls, requiredMinTiDBVersion, requiredMaxTiDBVersion) + require.Error(t, err) + require.Regexp(t, "^TiDB version too new", err.Error()) version = "5.7.25-TiDB-v6.0.0-beta" - c.Assert(checkTiDBVersionByTLS(ctx, tls, requiredMinTiDBVersion, requiredMaxTiDBVersion), ErrorMatches, "TiDB version too new.*") + err = checkTiDBVersionByTLS(ctx, tls, requiredMinTiDBVersion, requiredMaxTiDBVersion) + require.Error(t, err) + require.Regexp(t, "^TiDB version too new", err.Error()) version = "5.7.25-TiDB-v1.0.0" - c.Assert(checkTiDBVersionByTLS(ctx, tls, requiredMinTiDBVersion, requiredMaxTiDBVersion), ErrorMatches, "TiDB version too old.*") + err = checkTiDBVersionByTLS(ctx, tls, requiredMinTiDBVersion, requiredMaxTiDBVersion) + require.Error(t, err) + require.Regexp(t, "^TiDB version too old", err.Error()) } diff --git a/br/pkg/lightning/backend/local/engine.go b/br/pkg/lightning/backend/local/engine.go new file mode 100644 index 0000000000000..a4ba47ac42a43 --- /dev/null +++ b/br/pkg/lightning/backend/local/engine.go @@ -0,0 +1,1474 @@ +// 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 local + +import ( + "bytes" + "container/heap" + "context" + "encoding/binary" + "encoding/json" + "fmt" + "io" + "os" + "path/filepath" + "sort" + "sync" + "time" + + "github.com/cockroachdb/pebble" + "github.com/cockroachdb/pebble/sstable" + "github.com/google/btree" + "github.com/google/uuid" + "github.com/pingcap/errors" + "github.com/pingcap/tidb/br/pkg/lightning/backend" + "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" + "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" + "github.com/pingcap/tidb/br/pkg/lightning/common" + "github.com/pingcap/tidb/br/pkg/lightning/errormanager" + "github.com/pingcap/tidb/br/pkg/lightning/log" + "github.com/pingcap/tidb/br/pkg/logutil" + "github.com/pingcap/tidb/br/pkg/membuf" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/util/hack" + "go.uber.org/atomic" + "go.uber.org/zap" + "golang.org/x/sync/errgroup" +) + +var ( + engineMetaKey = []byte{0, 'm', 'e', 't', 'a'} + normalIterStartKey = []byte{1} +) + +type importMutexState uint32 + +const ( + importMutexStateImport importMutexState = 1 << iota + importMutexStateClose + // importMutexStateReadLock is a special state because in this state we lock engine with read lock + // and add isImportingAtomic with this value. In other state, we directly store with the state value. + // so this must always the last value of this enum. + importMutexStateReadLock +) + +// engineMeta contains some field that is necessary to continue the engine restore/import process. +// These field should be written to disk when we update chunk checkpoint +type engineMeta struct { + TS uint64 `json:"ts"` + // Length is the number of KV pairs stored by the engine. + Length atomic.Int64 `json:"length"` + // TotalSize is the total pre-compressed KV byte size stored by engine. + TotalSize atomic.Int64 `json:"total_size"` + // Duplicates is the number of duplicates kv pairs detected when importing. Note that the value is + // probably larger than real value, because we may import same range more than once. For accurate + // information, you should iterate the duplicate db after import is finished. + Duplicates atomic.Int64 `json:"duplicates"` +} + +type syncedRanges struct { + sync.Mutex + ranges []Range +} + +func (r *syncedRanges) add(g Range) { + r.Lock() + r.ranges = append(r.ranges, g) + r.Unlock() +} + +func (r *syncedRanges) reset() { + r.Lock() + r.ranges = r.ranges[:0] + r.Unlock() +} + +type Engine struct { + engineMeta + closed atomic.Bool + db *pebble.DB + UUID uuid.UUID + localWriters sync.Map + + // isImportingAtomic is an atomic variable indicating whether this engine is importing. + // This should not be used as a "spin lock" indicator. + isImportingAtomic atomic.Uint32 + // flush and ingest sst hold the rlock, other operation hold the wlock. + mutex sync.RWMutex + + ctx context.Context + cancel context.CancelFunc + sstDir string + sstMetasChan chan metaOrFlush + ingestErr common.OnceError + wg sync.WaitGroup + sstIngester sstIngester + finishedRanges syncedRanges + + // sst seq lock + seqLock sync.Mutex + // seq number for incoming sst meta + nextSeq int32 + // max seq of sst metas ingested into pebble + finishedMetaSeq atomic.Int32 + + config backend.LocalEngineConfig + tableInfo *checkpoints.TidbTableInfo + + // total size of SST files waiting to be ingested + pendingFileSize atomic.Int64 + + // statistics for pebble kv iter. + importedKVSize atomic.Int64 + importedKVCount atomic.Int64 + + keyAdapter KeyAdapter + duplicateDetection bool + duplicateDB *pebble.DB + errorMgr *errormanager.ErrorManager +} + +func (e *Engine) setError(err error) { + if err != nil { + e.ingestErr.Set(err) + e.cancel() + } +} + +func (e *Engine) Close() error { + log.L().Debug("closing local engine", zap.Stringer("engine", e.UUID), zap.Stack("stack")) + if e.db == nil { + return nil + } + err := errors.Trace(e.db.Close()) + e.db = nil + return err +} + +// Cleanup remove meta and db files +func (e *Engine) Cleanup(dataDir string) error { + if err := os.RemoveAll(e.sstDir); err != nil { + return errors.Trace(err) + } + + dbPath := filepath.Join(dataDir, e.UUID.String()) + return os.RemoveAll(dbPath) +} + +// Exist checks if db folder existing (meta sometimes won't flush before lightning exit) +func (e *Engine) Exist(dataDir string) error { + dbPath := filepath.Join(dataDir, e.UUID.String()) + if _, err := os.Stat(dbPath); err != nil { + return err + } + return nil +} + +func isStateLocked(state importMutexState) bool { + return state&(importMutexStateClose|importMutexStateImport) != 0 +} + +func (e *Engine) isLocked() bool { + // the engine is locked only in import or close state. + return isStateLocked(importMutexState(e.isImportingAtomic.Load())) +} + +// rLock locks the local file with shard read state. Only used for flush and ingest SST files. +func (e *Engine) rLock() { + e.mutex.RLock() + e.isImportingAtomic.Add(uint32(importMutexStateReadLock)) +} + +func (e *Engine) rUnlock() { + if e == nil { + return + } + + e.isImportingAtomic.Sub(uint32(importMutexStateReadLock)) + e.mutex.RUnlock() +} + +// lock locks the local file for importing. +func (e *Engine) lock(state importMutexState) { + e.mutex.Lock() + e.isImportingAtomic.Store(uint32(state)) +} + +// lockUnless tries to lock the local file unless it is already locked into the state given by +// ignoreStateMask. Returns whether the lock is successful. +func (e *Engine) lockUnless(newState, ignoreStateMask importMutexState) bool { + curState := e.isImportingAtomic.Load() + if curState&uint32(ignoreStateMask) != 0 { + return false + } + e.lock(newState) + return true +} + +// tryRLock tries to read-lock the local file unless it is already write locked. +// Returns whether the lock is successful. +func (e *Engine) tryRLock() bool { + curState := e.isImportingAtomic.Load() + // engine is in import/close state. + if isStateLocked(importMutexState(curState)) { + return false + } + e.rLock() + return true +} + +func (e *Engine) unlock() { + if e == nil { + return + } + e.isImportingAtomic.Store(0) + e.mutex.Unlock() +} + +type rangeOffsets struct { + Size uint64 + Keys uint64 +} + +type rangeProperty struct { + Key []byte + rangeOffsets +} + +func (r *rangeProperty) Less(than btree.Item) bool { + ta := than.(*rangeProperty) + return bytes.Compare(r.Key, ta.Key) < 0 +} + +var _ btree.Item = &rangeProperty{} + +type rangeProperties []rangeProperty + +func decodeRangeProperties(data []byte) (rangeProperties, error) { + r := make(rangeProperties, 0, 16) + for len(data) > 0 { + if len(data) < 4 { + return nil, io.ErrUnexpectedEOF + } + keyLen := int(binary.BigEndian.Uint32(data[:4])) + data = data[4:] + if len(data) < keyLen+8*2 { + return nil, io.ErrUnexpectedEOF + } + key := data[:keyLen] + data = data[keyLen:] + size := binary.BigEndian.Uint64(data[:8]) + keys := binary.BigEndian.Uint64(data[8:]) + data = data[16:] + r = append(r, rangeProperty{Key: key, rangeOffsets: rangeOffsets{Size: size, Keys: keys}}) + } + + return r, nil +} + +func (r rangeProperties) Encode() []byte { + b := make([]byte, 0, 1024) + idx := 0 + for _, p := range r { + b = append(b, 0, 0, 0, 0) + binary.BigEndian.PutUint32(b[idx:], uint32(len(p.Key))) + idx += 4 + b = append(b, p.Key...) + idx += len(p.Key) + + b = append(b, 0, 0, 0, 0, 0, 0, 0, 0) + binary.BigEndian.PutUint64(b[idx:], p.Size) + idx += 8 + + b = append(b, 0, 0, 0, 0, 0, 0, 0, 0) + binary.BigEndian.PutUint64(b[idx:], p.Keys) + idx += 8 + } + return b +} + +func (r rangeProperties) get(key []byte) rangeOffsets { + idx := sort.Search(len(r), func(i int) bool { + return bytes.Compare(r[i].Key, key) >= 0 + }) + return r[idx].rangeOffsets +} + +type RangePropertiesCollector struct { + props rangeProperties + lastOffsets rangeOffsets + lastKey []byte + currentOffsets rangeOffsets + propSizeIdxDistance uint64 + propKeysIdxDistance uint64 +} + +func newRangePropertiesCollector() pebble.TablePropertyCollector { + return &RangePropertiesCollector{ + props: make([]rangeProperty, 0, 1024), + propSizeIdxDistance: defaultPropSizeIndexDistance, + propKeysIdxDistance: defaultPropKeysIndexDistance, + } +} + +func (c *RangePropertiesCollector) sizeInLastRange() uint64 { + return c.currentOffsets.Size - c.lastOffsets.Size +} + +func (c *RangePropertiesCollector) keysInLastRange() uint64 { + return c.currentOffsets.Keys - c.lastOffsets.Keys +} + +func (c *RangePropertiesCollector) insertNewPoint(key []byte) { + c.lastOffsets = c.currentOffsets + c.props = append(c.props, rangeProperty{Key: append([]byte{}, key...), rangeOffsets: c.currentOffsets}) +} + +// Add implements `pebble.TablePropertyCollector`. +// Add implements `TablePropertyCollector.Add`. +func (c *RangePropertiesCollector) Add(key pebble.InternalKey, value []byte) error { + c.currentOffsets.Size += uint64(len(value)) + uint64(len(key.UserKey)) + c.currentOffsets.Keys++ + if len(c.lastKey) == 0 || c.sizeInLastRange() >= c.propSizeIdxDistance || + c.keysInLastRange() >= c.propKeysIdxDistance { + c.insertNewPoint(key.UserKey) + } + c.lastKey = append(c.lastKey[:0], key.UserKey...) + return nil +} + +func (c *RangePropertiesCollector) Finish(userProps map[string]string) error { + if c.sizeInLastRange() > 0 || c.keysInLastRange() > 0 { + c.insertNewPoint(c.lastKey) + } + + userProps[propRangeIndex] = string(c.props.Encode()) + return nil +} + +func (c *RangePropertiesCollector) Name() string { + return propRangeIndex +} + +type sizeProperties struct { + totalSize uint64 + indexHandles *btree.BTree +} + +func newSizeProperties() *sizeProperties { + return &sizeProperties{indexHandles: btree.New(32)} +} + +func (s *sizeProperties) add(item *rangeProperty) { + if old := s.indexHandles.ReplaceOrInsert(item); old != nil { + o := old.(*rangeProperty) + item.Keys += o.Keys + item.Size += o.Size + } +} + +func (s *sizeProperties) addAll(props rangeProperties) { + prevRange := rangeOffsets{} + for _, r := range props { + s.add(&rangeProperty{ + Key: r.Key, + rangeOffsets: rangeOffsets{Keys: r.Keys - prevRange.Keys, Size: r.Size - prevRange.Size}, + }) + prevRange = r.rangeOffsets + } + if len(props) > 0 { + s.totalSize = props[len(props)-1].Size + } +} + +// iter the tree until f return false +func (s *sizeProperties) iter(f func(p *rangeProperty) bool) { + s.indexHandles.Ascend(func(i btree.Item) bool { + prop := i.(*rangeProperty) + return f(prop) + }) +} + +func (e *Engine) getSizeProperties() (*sizeProperties, error) { + sstables, err := e.db.SSTables(pebble.WithProperties()) + if err != nil { + log.L().Warn("get table properties failed", zap.Stringer("engine", e.UUID), log.ShortError(err)) + return nil, errors.Trace(err) + } + + sizeProps := newSizeProperties() + for _, level := range sstables { + for _, info := range level { + if prop, ok := info.Properties.UserProperties[propRangeIndex]; ok { + data := hack.Slice(prop) + rangeProps, err := decodeRangeProperties(data) + if err != nil { + log.L().Warn("decodeRangeProperties failed", zap.Stringer("engine", e.UUID), + zap.Stringer("fileNum", info.FileNum), log.ShortError(err)) + return nil, errors.Trace(err) + } + if e.duplicateDetection { + newRangeProps := make(rangeProperties, 0, len(rangeProps)) + for _, p := range rangeProps { + if !bytes.Equal(p.Key, engineMetaKey) { + p.Key, _, _, err = e.keyAdapter.Decode(nil, p.Key) + if err != nil { + log.L().Warn( + "decodeRangeProperties failed because the props key is invalid", + zap.Stringer("engine", e.UUID), + zap.Stringer("fileNum", info.FileNum), + zap.Binary("key", p.Key), + ) + return nil, errors.Trace(err) + } + newRangeProps = append(newRangeProps, p) + } + } + rangeProps = newRangeProps + } + sizeProps.addAll(rangeProps) + } + } + } + + return sizeProps, nil +} + +func (e *Engine) getEngineFileSize() backend.EngineFileSize { + metrics := e.db.Metrics() + total := metrics.Total() + var memSize int64 + e.localWriters.Range(func(k, v interface{}) bool { + w := k.(*Writer) + if w.writer != nil { + memSize += int64(w.writer.writer.EstimatedSize()) + } else { + // if kvs are still in memory, only calculate half of the total size + // in our tests, SST file size is about 50% of the raw kv size + memSize += w.batchSize / 2 + } + + return true + }) + + pendingSize := e.pendingFileSize.Load() + // TODO: should also add the in-processing compaction sst writer size into MemSize + return backend.EngineFileSize{ + UUID: e.UUID, + DiskSize: total.Size + pendingSize, + MemSize: memSize, + IsImporting: e.isLocked(), + } +} + +// either a sstMeta or a flush message +type metaOrFlush struct { + meta *sstMeta + flushCh chan struct{} +} + +type metaSeq struct { + // the sequence for this flush message, a flush call can return only if + // all the other flush will lower `flushSeq` are done + flushSeq int32 + // the max sstMeta sequence number in this flush, after the flush is done (all SSTs are ingested), + // we can save chunks will a lower meta sequence number safely. + metaSeq int32 +} + +type metaSeqHeap struct { + arr []metaSeq +} + +func (h *metaSeqHeap) Len() int { + return len(h.arr) +} + +func (h *metaSeqHeap) Less(i, j int) bool { + return h.arr[i].flushSeq < h.arr[j].flushSeq +} + +func (h *metaSeqHeap) Swap(i, j int) { + h.arr[i], h.arr[j] = h.arr[j], h.arr[i] +} + +func (h *metaSeqHeap) Push(x interface{}) { + h.arr = append(h.arr, x.(metaSeq)) +} + +func (h *metaSeqHeap) Pop() interface{} { + item := h.arr[len(h.arr)-1] + h.arr = h.arr[:len(h.arr)-1] + return item +} + +func (e *Engine) ingestSSTLoop() { + defer e.wg.Done() + + type flushSeq struct { + seq int32 + ch chan struct{} + } + + seq := atomic.NewInt32(0) + finishedSeq := atomic.NewInt32(0) + var seqLock sync.Mutex + // a flush is finished iff all the compaction&ingest tasks with a lower seq number are finished. + flushQueue := make([]flushSeq, 0) + // inSyncSeqs is a heap that stores all the finished compaction tasks whose seq is bigger than `finishedSeq + 1` + // this mean there are still at lease one compaction task with a lower seq unfinished. + inSyncSeqs := &metaSeqHeap{arr: make([]metaSeq, 0)} + + type metaAndSeq struct { + metas []*sstMeta + seq int32 + } + + concurrency := e.config.CompactConcurrency + // when compaction is disabled, ingest is an serial action, so 1 routine is enough + if !e.config.Compact { + concurrency = 1 + } + metaChan := make(chan metaAndSeq, concurrency) + for i := 0; i < concurrency; i++ { + e.wg.Add(1) + go func() { + defer e.wg.Done() + defer func() { + if e.ingestErr.Get() != nil { + seqLock.Lock() + for _, f := range flushQueue { + f.ch <- struct{}{} + } + flushQueue = flushQueue[:0] + seqLock.Unlock() + } + }() + for { + select { + case <-e.ctx.Done(): + return + case metas, ok := <-metaChan: + if !ok { + return + } + ingestMetas := metas.metas + if e.config.Compact { + newMeta, err := e.sstIngester.mergeSSTs(metas.metas, e.sstDir) + if err != nil { + e.setError(err) + return + } + ingestMetas = []*sstMeta{newMeta} + } + // batchIngestSSTs will change ingestMetas' order, so we record the max seq here + metasMaxSeq := ingestMetas[len(ingestMetas)-1].seq + + if err := e.batchIngestSSTs(ingestMetas); err != nil { + e.setError(err) + return + } + seqLock.Lock() + finSeq := finishedSeq.Load() + if metas.seq == finSeq+1 { + finSeq = metas.seq + finMetaSeq := metasMaxSeq + for len(inSyncSeqs.arr) > 0 { + if inSyncSeqs.arr[0].flushSeq == finSeq+1 { + finSeq++ + finMetaSeq = inSyncSeqs.arr[0].metaSeq + heap.Remove(inSyncSeqs, 0) + } else { + break + } + } + + var flushChans []chan struct{} + for _, seq := range flushQueue { + if seq.seq <= finSeq { + flushChans = append(flushChans, seq.ch) + } else { + break + } + } + flushQueue = flushQueue[len(flushChans):] + finishedSeq.Store(finSeq) + e.finishedMetaSeq.Store(finMetaSeq) + seqLock.Unlock() + for _, c := range flushChans { + c <- struct{}{} + } + } else { + heap.Push(inSyncSeqs, metaSeq{flushSeq: metas.seq, metaSeq: metasMaxSeq}) + seqLock.Unlock() + } + } + } + }() + } + + compactAndIngestSSTs := func(metas []*sstMeta) { + if len(metas) > 0 { + seqLock.Lock() + metaSeq := seq.Add(1) + seqLock.Unlock() + select { + case <-e.ctx.Done(): + case metaChan <- metaAndSeq{metas: metas, seq: metaSeq}: + } + } + } + + pendingMetas := make([]*sstMeta, 0, 16) + totalSize := int64(0) + metasTmp := make([]*sstMeta, 0) + addMetas := func() { + if len(metasTmp) == 0 { + return + } + metas := metasTmp + metasTmp = make([]*sstMeta, 0, len(metas)) + if !e.config.Compact { + compactAndIngestSSTs(metas) + return + } + for _, m := range metas { + if m.totalCount > 0 { + pendingMetas = append(pendingMetas, m) + totalSize += m.totalSize + if totalSize >= e.config.CompactThreshold { + compactMetas := pendingMetas + pendingMetas = make([]*sstMeta, 0, len(pendingMetas)) + totalSize = 0 + compactAndIngestSSTs(compactMetas) + } + } + } + } +readMetaLoop: + for { + closed := false + select { + case <-e.ctx.Done(): + close(metaChan) + return + case m, ok := <-e.sstMetasChan: + if !ok { + closed = true + break + } + if m.flushCh != nil { + // meet a flush event, we should trigger a ingest task if there are pending metas, + // and then waiting for all the running flush tasks to be done. + if len(metasTmp) > 0 { + addMetas() + } + if len(pendingMetas) > 0 { + seqLock.Lock() + metaSeq := seq.Add(1) + flushQueue = append(flushQueue, flushSeq{ch: m.flushCh, seq: metaSeq}) + seqLock.Unlock() + select { + case metaChan <- metaAndSeq{metas: pendingMetas, seq: metaSeq}: + case <-e.ctx.Done(): + close(metaChan) + return + } + + pendingMetas = make([]*sstMeta, 0, len(pendingMetas)) + totalSize = 0 + } else { + // none remaining metas needed to be ingested + seqLock.Lock() + curSeq := seq.Load() + finSeq := finishedSeq.Load() + // if all pending SST files are written, directly do a db.Flush + if curSeq == finSeq { + seqLock.Unlock() + m.flushCh <- struct{}{} + } else { + // waiting for pending compaction tasks + flushQueue = append(flushQueue, flushSeq{ch: m.flushCh, seq: curSeq}) + seqLock.Unlock() + } + } + continue readMetaLoop + } + metasTmp = append(metasTmp, m.meta) + // try to drain all the sst meta from the chan to make sure all the SSTs are processed before handle a flush msg. + if len(e.sstMetasChan) > 0 { + continue readMetaLoop + } + + addMetas() + } + if closed { + compactAndIngestSSTs(pendingMetas) + close(metaChan) + return + } + } +} + +func (e *Engine) addSST(ctx context.Context, m *sstMeta) (int32, error) { + // set pending size after SST file is generated + e.pendingFileSize.Add(m.fileSize) + // make sure sstMeta is sent into the chan in order + e.seqLock.Lock() + defer e.seqLock.Unlock() + e.nextSeq++ + seq := e.nextSeq + m.seq = seq + select { + case e.sstMetasChan <- metaOrFlush{meta: m}: + case <-ctx.Done(): + return 0, ctx.Err() + case <-e.ctx.Done(): + } + return seq, e.ingestErr.Get() +} + +func (e *Engine) batchIngestSSTs(metas []*sstMeta) error { + if len(metas) == 0 { + return nil + } + sort.Slice(metas, func(i, j int) bool { + return bytes.Compare(metas[i].minKey, metas[j].minKey) < 0 + }) + + metaLevels := make([][]*sstMeta, 0) + for _, meta := range metas { + inserted := false + for i, l := range metaLevels { + if bytes.Compare(l[len(l)-1].maxKey, meta.minKey) >= 0 { + continue + } + metaLevels[i] = append(l, meta) + inserted = true + break + } + if !inserted { + metaLevels = append(metaLevels, []*sstMeta{meta}) + } + } + + for _, l := range metaLevels { + if err := e.ingestSSTs(l); err != nil { + return err + } + } + return nil +} + +func (e *Engine) ingestSSTs(metas []*sstMeta) error { + // use raw RLock to avoid change the lock state during flushing. + e.mutex.RLock() + defer e.mutex.RUnlock() + if e.closed.Load() { + return errorEngineClosed + } + totalSize := int64(0) + totalCount := int64(0) + fileSize := int64(0) + for _, m := range metas { + totalSize += m.totalSize + totalCount += m.totalCount + fileSize += m.fileSize + } + log.L().Info("write data to local DB", + zap.Int64("size", totalSize), + zap.Int64("kvs", totalCount), + zap.Int("files", len(metas)), + zap.Int64("sstFileSize", fileSize), + zap.String("file", metas[0].path), + logutil.Key("firstKey", metas[0].minKey), + logutil.Key("lastKey", metas[len(metas)-1].maxKey)) + if err := e.sstIngester.ingest(metas); err != nil { + return errors.Trace(err) + } + count := int64(0) + size := int64(0) + for _, m := range metas { + count += m.totalCount + size += m.totalSize + } + e.Length.Add(count) + e.TotalSize.Add(size) + return nil +} + +func (e *Engine) flushLocalWriters(parentCtx context.Context) error { + eg, ctx := errgroup.WithContext(parentCtx) + e.localWriters.Range(func(k, v interface{}) bool { + eg.Go(func() error { + w := k.(*Writer) + return w.flush(ctx) + }) + return true + }) + return eg.Wait() +} + +func (e *Engine) flushEngineWithoutLock(ctx context.Context) error { + if err := e.flushLocalWriters(ctx); err != nil { + return err + } + flushChan := make(chan struct{}, 1) + select { + case e.sstMetasChan <- metaOrFlush{flushCh: flushChan}: + case <-ctx.Done(): + return ctx.Err() + case <-e.ctx.Done(): + return e.ctx.Err() + } + + select { + case <-flushChan: + case <-ctx.Done(): + return ctx.Err() + case <-e.ctx.Done(): + return e.ctx.Err() + } + if err := e.ingestErr.Get(); err != nil { + return errors.Trace(err) + } + if err := e.saveEngineMeta(); err != nil { + return err + } + + flushFinishedCh, err := e.db.AsyncFlush() + if err != nil { + return errors.Trace(err) + } + select { + case <-flushFinishedCh: + return nil + case <-ctx.Done(): + return ctx.Err() + case <-e.ctx.Done(): + return e.ctx.Err() + } +} + +func saveEngineMetaToDB(meta *engineMeta, db *pebble.DB) error { + jsonBytes, err := json.Marshal(meta) + if err != nil { + return errors.Trace(err) + } + // note: we can't set Sync to true since we disabled WAL. + return db.Set(engineMetaKey, jsonBytes, &pebble.WriteOptions{Sync: false}) +} + +// saveEngineMeta saves the metadata about the DB into the DB itself. +// This method should be followed by a Flush to ensure the data is actually synchronized +func (e *Engine) saveEngineMeta() error { + log.L().Debug("save engine meta", zap.Stringer("uuid", e.UUID), zap.Int64("count", e.Length.Load()), + zap.Int64("size", e.TotalSize.Load())) + return errors.Trace(saveEngineMetaToDB(&e.engineMeta, e.db)) +} + +func (e *Engine) loadEngineMeta() error { + jsonBytes, closer, err := e.db.Get(engineMetaKey) + if err != nil { + if err == pebble.ErrNotFound { + log.L().Debug("local db missing engine meta", zap.Stringer("uuid", e.UUID), zap.Error(err)) + return nil + } + return err + } + defer closer.Close() + + if err = json.Unmarshal(jsonBytes, &e.engineMeta); err != nil { + log.L().Warn("local db failed to deserialize meta", zap.Stringer("uuid", e.UUID), zap.ByteString("content", jsonBytes), zap.Error(err)) + return err + } + log.L().Debug("load engine meta", zap.Stringer("uuid", e.UUID), zap.Int64("count", e.Length.Load()), + zap.Int64("size", e.TotalSize.Load())) + return nil +} + +// sortAndMergeRanges sort the ranges and merge range that overlaps with each other into a single range. +func sortAndMergeRanges(ranges []Range) []Range { + if len(ranges) == 0 { + return ranges + } + + sort.Slice(ranges, func(i, j int) bool { + return bytes.Compare(ranges[i].start, ranges[j].start) < 0 + }) + + curEnd := ranges[0].end + i := 0 + for j := 1; j < len(ranges); j++ { + if bytes.Compare(curEnd, ranges[j].start) >= 0 { + if bytes.Compare(curEnd, ranges[j].end) < 0 { + curEnd = ranges[j].end + } + } else { + ranges[i].end = curEnd + i++ + ranges[i].start = ranges[j].start + curEnd = ranges[j].end + } + } + ranges[i].end = curEnd + return ranges[:i+1] +} + +func filterOverlapRange(ranges []Range, finishedRanges []Range) []Range { + if len(ranges) == 0 || len(finishedRanges) == 0 { + return ranges + } + + result := make([]Range, 0) + for _, r := range ranges { + start := r.start + end := r.end + for len(finishedRanges) > 0 && bytes.Compare(finishedRanges[0].start, end) < 0 { + fr := finishedRanges[0] + if bytes.Compare(fr.start, start) > 0 { + result = append(result, Range{start: start, end: fr.start}) + } + if bytes.Compare(fr.end, start) > 0 { + start = fr.end + } + if bytes.Compare(fr.end, end) > 0 { + break + } + finishedRanges = finishedRanges[1:] + } + if bytes.Compare(start, end) < 0 { + result = append(result, Range{start: start, end: end}) + } + } + return result +} + +func (e *Engine) unfinishedRanges(ranges []Range) []Range { + e.finishedRanges.Lock() + defer e.finishedRanges.Unlock() + + e.finishedRanges.ranges = sortAndMergeRanges(e.finishedRanges.ranges) + + return filterOverlapRange(ranges, e.finishedRanges.ranges) +} + +type sstMeta struct { + path string + minKey []byte + maxKey []byte + totalSize int64 + totalCount int64 + // used for calculate disk-quota + fileSize int64 + seq int32 +} + +type Writer struct { + sync.Mutex + engine *Engine + memtableSizeLimit int64 + + // if the KVs are append in order, we can directly write the into SST file, + // else we must first store them in writeBatch and then batch flush into SST file. + isKVSorted bool + writer *sstWriter + + // bytes buffer for writeBatch + kvBuffer *membuf.Buffer + writeBatch []common.KvPair + // if the kvs in writeBatch are in order, we can avoid doing a `sort.Slice` which + // is quite slow. in our bench, the sort operation eats about 5% of total CPU + isWriteBatchSorted bool + + batchCount int + batchSize int64 + totalSize int64 + totalCount int64 + + lastMetaSeq int32 +} + +func (w *Writer) appendRowsSorted(kvs []common.KvPair) error { + if w.writer == nil { + writer, err := w.createSSTWriter() + if err != nil { + return errors.Trace(err) + } + w.writer = writer + w.writer.minKey = append([]byte{}, kvs[0].Key...) + } + + totalKeyLen := 0 + for i := 0; i < len(kvs); i++ { + totalKeyLen += w.engine.keyAdapter.EncodedLen(kvs[i].Key) + } + buf := make([]byte, totalKeyLen) + encodedKvs := make([]common.KvPair, len(kvs)) + for i := 0; i < len(kvs); i++ { + encodedKey := w.engine.keyAdapter.Encode(buf, kvs[i].Key, kvs[i].RowID, kvs[i].Offset) + buf = buf[len(encodedKey):] + encodedKvs[i] = common.KvPair{Key: encodedKey, Val: kvs[i].Val} + w.batchSize += int64(len(encodedKvs[i].Key) + len(encodedKvs[i].Val)) + } + + w.batchCount += len(encodedKvs) + w.totalCount += int64(len(encodedKvs)) + return w.writer.writeKVs(encodedKvs) +} + +func (w *Writer) appendRowsUnsorted(ctx context.Context, kvs []common.KvPair) error { + l := len(w.writeBatch) + cnt := w.batchCount + var lastKey []byte + if cnt > 0 { + lastKey = w.writeBatch[cnt-1].Key + } + for _, pair := range kvs { + if w.isWriteBatchSorted && bytes.Compare(lastKey, pair.Key) > 0 { + w.isWriteBatchSorted = false + } + lastKey = pair.Key + w.batchSize += int64(len(pair.Key) + len(pair.Val)) + buf := w.kvBuffer.AllocBytes(w.engine.keyAdapter.EncodedLen(pair.Key)) + key := w.engine.keyAdapter.Encode(buf, pair.Key, pair.RowID, pair.Offset) + val := w.kvBuffer.AddBytes(pair.Val) + if cnt < l { + w.writeBatch[cnt].Key = key + w.writeBatch[cnt].Val = val + } else { + w.writeBatch = append(w.writeBatch, common.KvPair{Key: key, Val: val}) + } + cnt++ + } + w.batchCount = cnt + + if w.batchSize > w.memtableSizeLimit { + if err := w.flushKVs(ctx); err != nil { + return err + } + } + w.totalCount += int64(len(kvs)) + return nil +} + +func (w *Writer) AppendRows(ctx context.Context, tableName string, columnNames []string, rows kv.Rows) error { + kvs := kv.KvPairsFromRows(rows) + if len(kvs) == 0 { + return nil + } + + if w.engine.closed.Load() { + return errorEngineClosed + } + + w.Lock() + defer w.Unlock() + + // if chunk has _tidb_rowid field, we can't ensure that the rows are sorted. + if w.isKVSorted && w.writer == nil { + for _, c := range columnNames { + if c == model.ExtraHandleName.L { + w.isKVSorted = false + } + } + } + + if w.isKVSorted { + return w.appendRowsSorted(kvs) + } + return w.appendRowsUnsorted(ctx, kvs) +} + +func (w *Writer) flush(ctx context.Context) error { + w.Lock() + defer w.Unlock() + if w.batchCount == 0 { + return nil + } + + w.totalSize += w.batchSize + if len(w.writeBatch) > 0 { + if err := w.flushKVs(ctx); err != nil { + return errors.Trace(err) + } + } + + if w.writer != nil { + meta, err := w.writer.close() + if err != nil { + return errors.Trace(err) + } + w.writer = nil + w.batchCount = 0 + if meta != nil && meta.totalSize > 0 { + return w.addSST(ctx, meta) + } + } + + return nil +} + +type flushStatus struct { + local *Engine + seq int32 +} + +func (f flushStatus) Flushed() bool { + return f.seq <= f.local.finishedMetaSeq.Load() +} + +func (w *Writer) Close(ctx context.Context) (backend.ChunkFlushStatus, error) { + defer w.kvBuffer.Destroy() + defer w.engine.localWriters.Delete(w) + err := w.flush(ctx) + // FIXME: in theory this line is useless, but In our benchmark with go1.15 + // this can resolve the memory consistently increasing issue. + // maybe this is a bug related to go GC mechanism. + w.writeBatch = nil + return flushStatus{local: w.engine, seq: w.lastMetaSeq}, err +} + +func (w *Writer) IsSynced() bool { + return w.batchCount == 0 && w.lastMetaSeq <= w.engine.finishedMetaSeq.Load() +} + +func (w *Writer) flushKVs(ctx context.Context) error { + writer, err := w.createSSTWriter() + if err != nil { + return errors.Trace(err) + } + if !w.isWriteBatchSorted { + sort.Slice(w.writeBatch[:w.batchCount], func(i, j int) bool { + return bytes.Compare(w.writeBatch[i].Key, w.writeBatch[j].Key) < 0 + }) + w.isWriteBatchSorted = true + } + + writer.minKey = append(writer.minKey[:0], w.writeBatch[0].Key...) + err = writer.writeKVs(w.writeBatch[:w.batchCount]) + if err != nil { + return errors.Trace(err) + } + meta, err := writer.close() + if err != nil { + return errors.Trace(err) + } + err = w.addSST(ctx, meta) + if err != nil { + return errors.Trace(err) + } + + w.totalSize += w.batchSize + w.batchSize = 0 + w.batchCount = 0 + w.kvBuffer.Reset() + return nil +} + +func (w *Writer) addSST(ctx context.Context, meta *sstMeta) error { + seq, err := w.engine.addSST(ctx, meta) + if err != nil { + return err + } + w.lastMetaSeq = seq + return nil +} + +func (w *Writer) createSSTWriter() (*sstWriter, error) { + path := filepath.Join(w.engine.sstDir, uuid.New().String()+".sst") + writer, err := newSSTWriter(path) + if err != nil { + return nil, err + } + sw := &sstWriter{sstMeta: &sstMeta{path: path}, writer: writer} + return sw, nil +} + +var errorUnorderedSSTInsertion = errors.New("inserting KVs into SST without order") + +type sstWriter struct { + *sstMeta + writer *sstable.Writer +} + +func newSSTWriter(path string) (*sstable.Writer, error) { + f, err := os.Create(path) + if err != nil { + return nil, errors.Trace(err) + } + writer := sstable.NewWriter(f, sstable.WriterOptions{ + TablePropertyCollectors: []func() pebble.TablePropertyCollector{ + newRangePropertiesCollector, + }, + BlockSize: 16 * 1024, + }) + return writer, nil +} + +func (sw *sstWriter) writeKVs(kvs []common.KvPair) error { + if len(kvs) == 0 { + return nil + } + + if bytes.Compare(kvs[0].Key, sw.maxKey) <= 0 { + return errorUnorderedSSTInsertion + } + + internalKey := sstable.InternalKey{ + Trailer: uint64(sstable.InternalKeyKindSet), + } + var lastKey []byte + for _, p := range kvs { + if bytes.Equal(p.Key, lastKey) { + log.L().Warn("duplicated key found, skip write", logutil.Key("key", p.Key)) + continue + } + internalKey.UserKey = p.Key + if err := sw.writer.Add(internalKey, p.Val); err != nil { + return errors.Trace(err) + } + sw.totalSize += int64(len(p.Key)) + int64(len(p.Val)) + } + sw.totalCount += int64(len(kvs)) + sw.maxKey = append(sw.maxKey[:0], kvs[len(kvs)-1].Key...) + return nil +} + +func (sw *sstWriter) close() (*sstMeta, error) { + if err := sw.writer.Close(); err != nil { + return nil, errors.Trace(err) + } + meta, err := sw.writer.Metadata() + if err != nil { + return nil, errors.Trace(err) + } + sw.fileSize = int64(meta.Size) + return sw.sstMeta, nil +} + +type sstIter struct { + name string + key []byte + val []byte + iter sstable.Iterator + reader *sstable.Reader + valid bool +} + +func (i *sstIter) Close() error { + if err := i.iter.Close(); err != nil { + return errors.Trace(err) + } + err := i.reader.Close() + return errors.Trace(err) +} + +type sstIterHeap struct { + iters []*sstIter +} + +func (h *sstIterHeap) Len() int { + return len(h.iters) +} + +func (h *sstIterHeap) Less(i, j int) bool { + return bytes.Compare(h.iters[i].key, h.iters[j].key) < 0 +} + +func (h *sstIterHeap) Swap(i, j int) { + h.iters[i], h.iters[j] = h.iters[j], h.iters[i] +} + +func (h *sstIterHeap) Push(x interface{}) { + h.iters = append(h.iters, x.(*sstIter)) +} + +func (h *sstIterHeap) Pop() interface{} { + item := h.iters[len(h.iters)-1] + h.iters = h.iters[:len(h.iters)-1] + return item +} + +func (h *sstIterHeap) Next() ([]byte, []byte, error) { + for { + if len(h.iters) == 0 { + return nil, nil, nil + } + + iter := h.iters[0] + if iter.valid { + iter.valid = false + return iter.key, iter.val, iter.iter.Error() + } + + var k *pebble.InternalKey + k, iter.val = iter.iter.Next() + if k != nil { + iter.key = k.UserKey + iter.valid = true + heap.Fix(h, 0) + } else { + err := iter.Close() + heap.Remove(h, 0) + if err != nil { + return nil, nil, errors.Trace(err) + } + } + } +} + +// sstIngester is a interface used to merge and ingest SST files. +// it's a interface mainly used for test convenience +type sstIngester interface { + mergeSSTs(metas []*sstMeta, dir string) (*sstMeta, error) + ingest([]*sstMeta) error +} + +type dbSSTIngester struct { + e *Engine +} + +func (i dbSSTIngester) mergeSSTs(metas []*sstMeta, dir string) (*sstMeta, error) { + if len(metas) == 0 { + return nil, errors.New("sst metas is empty") + } else if len(metas) == 1 { + return metas[0], nil + } + + start := time.Now() + newMeta := &sstMeta{ + seq: metas[len(metas)-1].seq, + } + mergeIter := &sstIterHeap{ + iters: make([]*sstIter, 0, len(metas)), + } + + for _, p := range metas { + f, err := os.Open(p.path) + if err != nil { + return nil, errors.Trace(err) + } + reader, err := sstable.NewReader(f, sstable.ReaderOptions{}) + if err != nil { + return nil, errors.Trace(err) + } + iter, err := reader.NewIter(nil, nil) + if err != nil { + return nil, errors.Trace(err) + } + key, val := iter.Next() + if key == nil { + continue + } + if iter.Error() != nil { + return nil, errors.Trace(iter.Error()) + } + mergeIter.iters = append(mergeIter.iters, &sstIter{ + name: p.path, + iter: iter, + key: key.UserKey, + val: val, + reader: reader, + valid: true, + }) + newMeta.totalSize += p.totalSize + newMeta.totalCount += p.totalCount + } + heap.Init(mergeIter) + + name := filepath.Join(dir, fmt.Sprintf("%s.sst", uuid.New())) + writer, err := newSSTWriter(name) + if err != nil { + return nil, errors.Trace(err) + } + newMeta.path = name + + internalKey := sstable.InternalKey{ + Trailer: uint64(sstable.InternalKeyKindSet), + } + key, val, err := mergeIter.Next() + if err != nil { + return nil, err + } + if key == nil { + return nil, errors.New("all ssts are empty!") + } + newMeta.minKey = append(newMeta.minKey[:0], key...) + lastKey := make([]byte, 0) + for { + if bytes.Equal(lastKey, key) { + log.L().Warn("duplicated key found, skipped", zap.Binary("key", lastKey)) + newMeta.totalCount-- + newMeta.totalSize -= int64(len(key) + len(val)) + + goto nextKey + } + internalKey.UserKey = key + err = writer.Add(internalKey, val) + if err != nil { + return nil, err + } + lastKey = append(lastKey[:0], key...) + nextKey: + key, val, err = mergeIter.Next() + if err != nil { + return nil, err + } + if key == nil { + break + } + } + err = writer.Close() + if err != nil { + return nil, errors.Trace(err) + } + meta, err := writer.Metadata() + if err != nil { + return nil, errors.Trace(err) + } + newMeta.maxKey = lastKey + newMeta.fileSize = int64(meta.Size) + + dur := time.Since(start) + log.L().Info("compact sst", zap.Int("fileCount", len(metas)), zap.Int64("size", newMeta.totalSize), + zap.Int64("count", newMeta.totalCount), zap.Duration("cost", dur), zap.String("file", name)) + + // async clean raw SSTs. + go func() { + totalSize := int64(0) + for _, m := range metas { + totalSize += m.fileSize + if err := os.Remove(m.path); err != nil { + log.L().Warn("async cleanup sst file failed", zap.Error(err)) + } + } + // decrease the pending size after clean up + i.e.pendingFileSize.Sub(totalSize) + }() + + return newMeta, err +} + +func (i dbSSTIngester) ingest(metas []*sstMeta) error { + if len(metas) == 0 { + return nil + } + paths := make([]string, 0, len(metas)) + for _, m := range metas { + paths = append(paths, m.path) + } + return i.e.db.Ingest(paths) +} diff --git a/br/pkg/lightning/backend/local/iterator.go b/br/pkg/lightning/backend/local/iterator.go index 8a04260a9ab0d..0495c6f4075a7 100644 --- a/br/pkg/lightning/backend/local/iterator.go +++ b/br/pkg/lightning/backend/local/iterator.go @@ -55,7 +55,7 @@ type duplicateIter struct { nextKey []byte err error - engineFile *File + engine *Engine keyAdapter KeyAdapter writeBatch *pebble.Batch writeBatchSize int64 @@ -100,7 +100,7 @@ func (d *duplicateIter) flush() { } func (d *duplicateIter) record(key []byte, val []byte) { - d.engineFile.Duplicates.Inc() + d.engine.Duplicates.Inc() d.err = d.writeBatch.Set(key, val, nil) if d.err != nil { return @@ -170,7 +170,7 @@ func (d *duplicateIter) OpType() sst.Pair_OP { var _ kv.Iter = &duplicateIter{} -func newDuplicateIter(ctx context.Context, engineFile *File, opts *pebble.IterOptions) kv.Iter { +func newDuplicateIter(ctx context.Context, engine *Engine, opts *pebble.IterOptions) kv.Iter { newOpts := &pebble.IterOptions{TableFilter: opts.TableFilter} if len(opts.LowerBound) > 0 { newOpts.LowerBound = codec.EncodeBytes(nil, opts.LowerBound) @@ -179,27 +179,27 @@ func newDuplicateIter(ctx context.Context, engineFile *File, opts *pebble.IterOp newOpts.UpperBound = codec.EncodeBytes(nil, opts.UpperBound) } logger := log.With( - zap.String("table", common.UniqueTable(engineFile.tableInfo.DB, engineFile.tableInfo.Name)), - zap.Int64("tableID", engineFile.tableInfo.ID), - zap.Stringer("engineUUID", engineFile.UUID)) + zap.String("table", common.UniqueTable(engine.tableInfo.DB, engine.tableInfo.Name)), + zap.Int64("tableID", engine.tableInfo.ID), + zap.Stringer("engineUUID", engine.UUID)) return &duplicateIter{ ctx: ctx, - iter: engineFile.db.NewIter(newOpts), - engineFile: engineFile, - keyAdapter: engineFile.keyAdapter, - writeBatch: engineFile.duplicateDB.NewBatch(), + iter: engine.db.NewIter(newOpts), + engine: engine, + keyAdapter: engine.keyAdapter, + writeBatch: engine.duplicateDB.NewBatch(), logger: logger, } } -func newKeyIter(ctx context.Context, engineFile *File, opts *pebble.IterOptions) kv.Iter { +func newKVIter(ctx context.Context, engine *Engine, opts *pebble.IterOptions) kv.Iter { if bytes.Compare(opts.LowerBound, normalIterStartKey) < 0 { newOpts := *opts newOpts.LowerBound = normalIterStartKey opts = &newOpts } - if !engineFile.duplicateDetection { - return pebbleIter{Iterator: engineFile.db.NewIter(opts)} + if !engine.duplicateDetection { + return pebbleIter{Iterator: engine.db.NewIter(opts)} } - return newDuplicateIter(ctx, engineFile, opts) + return newDuplicateIter(ctx, engine, opts) } diff --git a/br/pkg/lightning/backend/local/iterator_test.go b/br/pkg/lightning/backend/local/iterator_test.go index 83ca2365fa2cc..864d504844da1 100644 --- a/br/pkg/lightning/backend/local/iterator_test.go +++ b/br/pkg/lightning/backend/local/iterator_test.go @@ -130,7 +130,7 @@ func (s *iteratorSuite) TestDuplicateIterator(c *C) { duplicateDB, err := pebble.Open(filepath.Join(storeDir, "duplicates"), &pebble.Options{}) c.Assert(err, IsNil) - engineFile := &File{ + engine := &Engine{ ctx: context.Background(), db: db, keyAdapter: keyAdapter, @@ -140,7 +140,7 @@ func (s *iteratorSuite) TestDuplicateIterator(c *C) { Name: "name", }, } - iter := newDuplicateIter(context.Background(), engineFile, &pebble.IterOptions{}) + iter := newDuplicateIter(context.Background(), engine, &pebble.IterOptions{}) sort.Slice(pairs, func(i, j int) bool { key1 := keyAdapter.Encode(nil, pairs[i].Key, pairs[i].RowID, pairs[i].Offset) key2 := keyAdapter.Encode(nil, pairs[j].Key, pairs[j].RowID, pairs[j].Offset) @@ -167,7 +167,7 @@ func (s *iteratorSuite) TestDuplicateIterator(c *C) { c.Assert(iter.Error(), IsNil) c.Assert(len(uniqueKeys), Equals, 0) c.Assert(iter.Close(), IsNil) - c.Assert(engineFile.Close(), IsNil) + c.Assert(engine.Close(), IsNil) // Check duplicates detected by duplicate iterator. iter = pebbleIter{Iterator: duplicateDB.NewIter(&pebble.IterOptions{})} @@ -241,7 +241,7 @@ func (s *iteratorSuite) TestDuplicateIterSeek(c *C) { duplicateDB, err := pebble.Open(filepath.Join(storeDir, "duplicates"), &pebble.Options{}) c.Assert(err, IsNil) - engineFile := &File{ + engine := &Engine{ ctx: context.Background(), db: db, keyAdapter: keyAdapter, @@ -251,13 +251,13 @@ func (s *iteratorSuite) TestDuplicateIterSeek(c *C) { Name: "name", }, } - iter := newDuplicateIter(context.Background(), engineFile, &pebble.IterOptions{}) + iter := newDuplicateIter(context.Background(), engine, &pebble.IterOptions{}) c.Assert(iter.Seek([]byte{1, 2, 3, 1}), IsTrue) c.Assert(iter.Value(), BytesEquals, pairs[1].Val) c.Assert(iter.Next(), IsTrue) c.Assert(iter.Value(), BytesEquals, pairs[3].Val) c.Assert(iter.Close(), IsNil) - c.Assert(engineFile.Close(), IsNil) + c.Assert(engine.Close(), IsNil) c.Assert(duplicateDB.Close(), IsNil) } diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 0b86751ef5257..f4921da0fa55e 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -16,25 +16,18 @@ package local import ( "bytes" - "container/heap" "context" - "encoding/binary" - "encoding/json" "fmt" - "io" "math" "os" "path/filepath" - "sort" "strings" "sync" "time" "github.com/cockroachdb/pebble" - "github.com/cockroachdb/pebble/sstable" "github.com/coreos/go-semver/semver" "github.com/docker/go-units" - "github.com/google/btree" "github.com/google/uuid" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -44,7 +37,6 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/br/pkg/lightning/backend" "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" - "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/config" "github.com/pingcap/tidb/br/pkg/lightning/errormanager" @@ -65,7 +57,6 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/hack" "github.com/tikv/client-go/v2/oracle" tikvclient "github.com/tikv/client-go/v2/tikv" pd "github.com/tikv/pd/client" @@ -83,7 +74,6 @@ import ( const ( dialTimeout = 5 * time.Minute - bigValueSize = 1 << 16 // 64K maxRetryTimes = 5 defaultRetryBackoffTime = 3 * time.Second @@ -120,10 +110,8 @@ var ( errorEngineClosed = errors.New("engine is closed") ) -var ( - engineMetaKey = []byte{0, 'm', 'e', 't', 'a'} - normalIterStartKey = []byte{1} -) +// getImportClientFn is a variable alias for getImportClient used for unit test. +var getImportClientFn = getImportClient // Range record start and end key for localStoreDir.DB // so we can write it to tikv in streaming @@ -132,673 +120,8 @@ type Range struct { end []byte } -// localFileMeta contains some field that is necessary to continue the engine restore/import process. -// These field should be written to disk when we update chunk checkpoint -type localFileMeta struct { - TS uint64 `json:"ts"` - // Length is the number of KV pairs stored by the engine. - Length atomic.Int64 `json:"length"` - // TotalSize is the total pre-compressed KV byte size stored by engine. - TotalSize atomic.Int64 `json:"total_size"` - // Duplicates is the number of duplicates kv pairs detected when importing. Note that the value is - // probably larger than real value, because we may import same range more than once. For accurate - // information, you should iterate the duplicate db after import is finished. - Duplicates atomic.Int64 `json:"duplicates"` -} - -type importMutexState uint32 - -const ( - importMutexStateImport importMutexState = 1 << iota - importMutexStateClose - // importMutexStateReadLock is a special state because in this state we lock engine with read lock - // and add isImportingAtomic with this value. In other state, we directly store with the state value. - // so this must always the last value of this enum. - importMutexStateReadLock -) - -// either a sstMeta or a flush message -type metaOrFlush struct { - meta *sstMeta - flushCh chan struct{} -} - -type File struct { - localFileMeta - closed atomic.Bool - db *pebble.DB - UUID uuid.UUID - localWriters sync.Map - - // isImportingAtomic is an atomic variable indicating whether this engine is importing. - // This should not be used as a "spin lock" indicator. - isImportingAtomic atomic.Uint32 - // flush and ingest sst hold the rlock, other operation hold the wlock. - mutex sync.RWMutex - - ctx context.Context - cancel context.CancelFunc - sstDir string - sstMetasChan chan metaOrFlush - ingestErr common.OnceError - wg sync.WaitGroup - sstIngester sstIngester - finishedRanges syncedRanges - - // sst seq lock - seqLock sync.Mutex - // seq number for incoming sst meta - nextSeq int32 - // max seq of sst metas ingested into pebble - finishedMetaSeq atomic.Int32 - - config backend.LocalEngineConfig - tableInfo *checkpoints.TidbTableInfo - - // total size of SST files waiting to be ingested - pendingFileSize atomic.Int64 - - // statistics for pebble kv iter. - importedKVSize atomic.Int64 - importedKVCount atomic.Int64 - - keyAdapter KeyAdapter - duplicateDetection bool - duplicateDB *pebble.DB - errorMgr *errormanager.ErrorManager -} - -func (e *File) setError(err error) { - if err != nil { - e.ingestErr.Set(err) - e.cancel() - } -} - -func (e *File) Close() error { - log.L().Debug("closing local engine", zap.Stringer("engine", e.UUID), zap.Stack("stack")) - if e.db == nil { - return nil - } - err := errors.Trace(e.db.Close()) - e.db = nil - return err -} - -// Cleanup remove meta and db files -func (e *File) Cleanup(dataDir string) error { - if err := os.RemoveAll(e.sstDir); err != nil { - return errors.Trace(err) - } - - dbPath := filepath.Join(dataDir, e.UUID.String()) - return os.RemoveAll(dbPath) -} - -// Exist checks if db folder existing (meta sometimes won't flush before lightning exit) -func (e *File) Exist(dataDir string) error { - dbPath := filepath.Join(dataDir, e.UUID.String()) - if _, err := os.Stat(dbPath); err != nil { - return err - } - return nil -} - -func (e *File) getSizeProperties() (*sizeProperties, error) { - sstables, err := e.db.SSTables(pebble.WithProperties()) - if err != nil { - log.L().Warn("get table properties failed", zap.Stringer("engine", e.UUID), log.ShortError(err)) - return nil, errors.Trace(err) - } - - sizeProps := newSizeProperties() - for _, level := range sstables { - for _, info := range level { - if prop, ok := info.Properties.UserProperties[propRangeIndex]; ok { - data := hack.Slice(prop) - rangeProps, err := decodeRangeProperties(data) - if err != nil { - log.L().Warn("decodeRangeProperties failed", zap.Stringer("engine", e.UUID), - zap.Stringer("fileNum", info.FileNum), log.ShortError(err)) - return nil, errors.Trace(err) - } - if e.duplicateDetection { - newRangeProps := make(rangeProperties, 0, len(rangeProps)) - for _, p := range rangeProps { - if !bytes.Equal(p.Key, engineMetaKey) { - p.Key, _, _, err = e.keyAdapter.Decode(nil, p.Key) - if err != nil { - log.L().Warn( - "decodeRangeProperties failed because the props key is invalid", - zap.Stringer("engine", e.UUID), - zap.Stringer("fileNum", info.FileNum), - zap.Binary("key", p.Key), - ) - return nil, errors.Trace(err) - } - newRangeProps = append(newRangeProps, p) - } - } - rangeProps = newRangeProps - } - sizeProps.addAll(rangeProps) - } - } - } - - return sizeProps, nil -} - -func isStateLocked(state importMutexState) bool { - return state&(importMutexStateClose|importMutexStateImport) != 0 -} - -func (e *File) isLocked() bool { - // the engine is locked only in import or close state. - return isStateLocked(importMutexState(e.isImportingAtomic.Load())) -} - -func (e *File) getEngineFileSize() backend.EngineFileSize { - metrics := e.db.Metrics() - total := metrics.Total() - var memSize int64 - e.localWriters.Range(func(k, v interface{}) bool { - w := k.(*Writer) - if w.writer != nil { - memSize += int64(w.writer.writer.EstimatedSize()) - } else { - // if kvs are still in memory, only calculate half of the total size - // in our tests, SST file size is about 50% of the raw kv size - memSize += w.batchSize / 2 - } - - return true - }) - - pendingSize := e.pendingFileSize.Load() - // TODO: should also add the in-processing compaction sst writer size into MemSize - return backend.EngineFileSize{ - UUID: e.UUID, - DiskSize: total.Size + pendingSize, - MemSize: memSize, - IsImporting: e.isLocked(), - } -} - -// rLock locks the local file with shard read state. Only used for flush and ingest SST files. -func (e *File) rLock() { - e.mutex.RLock() - e.isImportingAtomic.Add(uint32(importMutexStateReadLock)) -} - -func (e *File) rUnlock() { - if e == nil { - return - } - - e.isImportingAtomic.Sub(uint32(importMutexStateReadLock)) - e.mutex.RUnlock() -} - -// lock locks the local file for importing. -func (e *File) lock(state importMutexState) { - e.mutex.Lock() - e.isImportingAtomic.Store(uint32(state)) -} - -// lockUnless tries to lock the local file unless it is already locked into the state given by -// ignoreStateMask. Returns whether the lock is successful. -func (e *File) lockUnless(newState, ignoreStateMask importMutexState) bool { - curState := e.isImportingAtomic.Load() - if curState&uint32(ignoreStateMask) != 0 { - return false - } - e.lock(newState) - return true -} - -// tryRLock tries to read-lock the local file unless it is already write locked. -// Returns whether the lock is successful. -func (e *File) tryRLock() bool { - curState := e.isImportingAtomic.Load() - // engine is in import/close state. - if isStateLocked(importMutexState(curState)) { - return false - } - e.rLock() - return true -} - -func (e *File) unlock() { - if e == nil { - return - } - e.isImportingAtomic.Store(0) - e.mutex.Unlock() -} - -type metaSeq struct { - // the sequence for this flush message, a flush call can return only if - // all the other flush will lower `flushSeq` are done - flushSeq int32 - // the max sstMeta sequence number in this flush, after the flush is done (all SSTs are ingested), - // we can save chunks will a lower meta sequence number safely. - metaSeq int32 -} - -type metaSeqHeap struct { - arr []metaSeq -} - -func (h *metaSeqHeap) Len() int { - return len(h.arr) -} - -func (h *metaSeqHeap) Less(i, j int) bool { - return h.arr[i].flushSeq < h.arr[j].flushSeq -} - -func (h *metaSeqHeap) Swap(i, j int) { - h.arr[i], h.arr[j] = h.arr[j], h.arr[i] -} - -func (h *metaSeqHeap) Push(x interface{}) { - h.arr = append(h.arr, x.(metaSeq)) -} - -func (h *metaSeqHeap) Pop() interface{} { - item := h.arr[len(h.arr)-1] - h.arr = h.arr[:len(h.arr)-1] - return item -} - -func (e *File) ingestSSTLoop() { - defer e.wg.Done() - - type flushSeq struct { - seq int32 - ch chan struct{} - } - - seq := atomic.NewInt32(0) - finishedSeq := atomic.NewInt32(0) - var seqLock sync.Mutex - // a flush is finished iff all the compaction&ingest tasks with a lower seq number are finished. - flushQueue := make([]flushSeq, 0) - // inSyncSeqs is a heap that stores all the finished compaction tasks whose seq is bigger than `finishedSeq + 1` - // this mean there are still at lease one compaction task with a lower seq unfinished. - inSyncSeqs := &metaSeqHeap{arr: make([]metaSeq, 0)} - - type metaAndSeq struct { - metas []*sstMeta - seq int32 - } - - concurrency := e.config.CompactConcurrency - // when compaction is disabled, ingest is an serial action, so 1 routine is enough - if !e.config.Compact { - concurrency = 1 - } - metaChan := make(chan metaAndSeq, concurrency) - for i := 0; i < concurrency; i++ { - e.wg.Add(1) - go func() { - defer e.wg.Done() - defer func() { - if e.ingestErr.Get() != nil { - seqLock.Lock() - for _, f := range flushQueue { - f.ch <- struct{}{} - } - flushQueue = flushQueue[:0] - seqLock.Unlock() - } - }() - for { - select { - case <-e.ctx.Done(): - return - case metas, ok := <-metaChan: - if !ok { - return - } - ingestMetas := metas.metas - if e.config.Compact { - newMeta, err := e.sstIngester.mergeSSTs(metas.metas, e.sstDir) - if err != nil { - e.setError(err) - return - } - ingestMetas = []*sstMeta{newMeta} - } - // batchIngestSSTs will change ingestMetas' order, so we record the max seq here - metasMaxSeq := ingestMetas[len(ingestMetas)-1].seq - - if err := e.batchIngestSSTs(ingestMetas); err != nil { - e.setError(err) - return - } - seqLock.Lock() - finSeq := finishedSeq.Load() - if metas.seq == finSeq+1 { - finSeq = metas.seq - finMetaSeq := metasMaxSeq - for len(inSyncSeqs.arr) > 0 { - if inSyncSeqs.arr[0].flushSeq == finSeq+1 { - finSeq++ - finMetaSeq = inSyncSeqs.arr[0].metaSeq - heap.Remove(inSyncSeqs, 0) - } else { - break - } - } - - var flushChans []chan struct{} - for _, seq := range flushQueue { - if seq.seq <= finSeq { - flushChans = append(flushChans, seq.ch) - } else { - break - } - } - flushQueue = flushQueue[len(flushChans):] - finishedSeq.Store(finSeq) - e.finishedMetaSeq.Store(finMetaSeq) - seqLock.Unlock() - for _, c := range flushChans { - c <- struct{}{} - } - } else { - heap.Push(inSyncSeqs, metaSeq{flushSeq: metas.seq, metaSeq: metasMaxSeq}) - seqLock.Unlock() - } - } - } - }() - } - - compactAndIngestSSTs := func(metas []*sstMeta) { - if len(metas) > 0 { - seqLock.Lock() - metaSeq := seq.Add(1) - seqLock.Unlock() - select { - case <-e.ctx.Done(): - case metaChan <- metaAndSeq{metas: metas, seq: metaSeq}: - } - } - } - - pendingMetas := make([]*sstMeta, 0, 16) - totalSize := int64(0) - metasTmp := make([]*sstMeta, 0) - addMetas := func() { - if len(metasTmp) == 0 { - return - } - metas := metasTmp - metasTmp = make([]*sstMeta, 0, len(metas)) - if !e.config.Compact { - compactAndIngestSSTs(metas) - return - } - for _, m := range metas { - if m.totalCount > 0 { - pendingMetas = append(pendingMetas, m) - totalSize += m.totalSize - if totalSize >= e.config.CompactThreshold { - compactMetas := pendingMetas - pendingMetas = make([]*sstMeta, 0, len(pendingMetas)) - totalSize = 0 - compactAndIngestSSTs(compactMetas) - } - } - } - } -readMetaLoop: - for { - closed := false - select { - case <-e.ctx.Done(): - close(metaChan) - return - case m, ok := <-e.sstMetasChan: - if !ok { - closed = true - break - } - if m.flushCh != nil { - // meet a flush event, we should trigger a ingest task if there are pending metas, - // and then waiting for all the running flush tasks to be done. - if len(metasTmp) > 0 { - addMetas() - } - if len(pendingMetas) > 0 { - seqLock.Lock() - metaSeq := seq.Add(1) - flushQueue = append(flushQueue, flushSeq{ch: m.flushCh, seq: metaSeq}) - seqLock.Unlock() - select { - case metaChan <- metaAndSeq{metas: pendingMetas, seq: metaSeq}: - case <-e.ctx.Done(): - close(metaChan) - return - } - - pendingMetas = make([]*sstMeta, 0, len(pendingMetas)) - totalSize = 0 - } else { - // none remaining metas needed to be ingested - seqLock.Lock() - curSeq := seq.Load() - finSeq := finishedSeq.Load() - // if all pending SST files are written, directly do a db.Flush - if curSeq == finSeq { - seqLock.Unlock() - m.flushCh <- struct{}{} - } else { - // waiting for pending compaction tasks - flushQueue = append(flushQueue, flushSeq{ch: m.flushCh, seq: curSeq}) - seqLock.Unlock() - } - } - continue readMetaLoop - } - metasTmp = append(metasTmp, m.meta) - // try to drain all the sst meta from the chan to make sure all the SSTs are processed before handle a flush msg. - if len(e.sstMetasChan) > 0 { - continue readMetaLoop - } - - addMetas() - } - if closed { - compactAndIngestSSTs(pendingMetas) - close(metaChan) - return - } - } -} - -func (e *File) addSST(ctx context.Context, m *sstMeta) (int32, error) { - // set pending size after SST file is generated - e.pendingFileSize.Add(m.fileSize) - // make sure sstMeta is sent into the chan in order - e.seqLock.Lock() - defer e.seqLock.Unlock() - e.nextSeq++ - seq := e.nextSeq - m.seq = seq - select { - case e.sstMetasChan <- metaOrFlush{meta: m}: - case <-ctx.Done(): - return 0, ctx.Err() - case <-e.ctx.Done(): - } - return seq, e.ingestErr.Get() -} - -func (e *File) batchIngestSSTs(metas []*sstMeta) error { - if len(metas) == 0 { - return nil - } - sort.Slice(metas, func(i, j int) bool { - return bytes.Compare(metas[i].minKey, metas[j].minKey) < 0 - }) - - metaLevels := make([][]*sstMeta, 0) - for _, meta := range metas { - inserted := false - for i, l := range metaLevels { - if bytes.Compare(l[len(l)-1].maxKey, meta.minKey) >= 0 { - continue - } - metaLevels[i] = append(l, meta) - inserted = true - break - } - if !inserted { - metaLevels = append(metaLevels, []*sstMeta{meta}) - } - } - - for _, l := range metaLevels { - if err := e.ingestSSTs(l); err != nil { - return err - } - } - return nil -} - -func (e *File) ingestSSTs(metas []*sstMeta) error { - // use raw RLock to avoid change the lock state during flushing. - e.mutex.RLock() - defer e.mutex.RUnlock() - if e.closed.Load() { - return errorEngineClosed - } - totalSize := int64(0) - totalCount := int64(0) - fileSize := int64(0) - for _, m := range metas { - totalSize += m.totalSize - totalCount += m.totalCount - fileSize += m.fileSize - } - log.L().Info("write data to local DB", - zap.Int64("size", totalSize), - zap.Int64("kvs", totalCount), - zap.Int("files", len(metas)), - zap.Int64("sstFileSize", fileSize), - zap.String("file", metas[0].path), - logutil.Key("firstKey", metas[0].minKey), - logutil.Key("lastKey", metas[len(metas)-1].maxKey)) - if err := e.sstIngester.ingest(metas); err != nil { - return errors.Trace(err) - } - count := int64(0) - size := int64(0) - for _, m := range metas { - count += m.totalCount - size += m.totalSize - } - e.Length.Add(count) - e.TotalSize.Add(size) - return nil -} - -func (e *File) flushLocalWriters(parentCtx context.Context) error { - eg, ctx := errgroup.WithContext(parentCtx) - e.localWriters.Range(func(k, v interface{}) bool { - eg.Go(func() error { - w := k.(*Writer) - return w.flush(ctx) - }) - return true - }) - return eg.Wait() -} - -func (e *File) flushEngineWithoutLock(ctx context.Context) error { - if err := e.flushLocalWriters(ctx); err != nil { - return err - } - flushChan := make(chan struct{}, 1) - select { - case e.sstMetasChan <- metaOrFlush{flushCh: flushChan}: - case <-ctx.Done(): - return ctx.Err() - case <-e.ctx.Done(): - return e.ctx.Err() - } - - select { - case <-flushChan: - case <-ctx.Done(): - return ctx.Err() - case <-e.ctx.Done(): - return e.ctx.Err() - } - if err := e.ingestErr.Get(); err != nil { - return errors.Trace(err) - } - if err := e.saveEngineMeta(); err != nil { - return err - } - - flushFinishedCh, err := e.db.AsyncFlush() - if err != nil { - return errors.Trace(err) - } - select { - case <-flushFinishedCh: - return nil - case <-ctx.Done(): - return ctx.Err() - case <-e.ctx.Done(): - return e.ctx.Err() - } -} - -func saveEngineMetaToDB(meta *localFileMeta, db *pebble.DB) error { - jsonBytes, err := json.Marshal(meta) - if err != nil { - return errors.Trace(err) - } - // note: we can't set Sync to true since we disabled WAL. - return db.Set(engineMetaKey, jsonBytes, &pebble.WriteOptions{Sync: false}) -} - -// saveEngineMeta saves the metadata about the DB into the DB itself. -// This method should be followed by a Flush to ensure the data is actually synchronized -func (e *File) saveEngineMeta() error { - log.L().Debug("save engine meta", zap.Stringer("uuid", e.UUID), zap.Int64("count", e.Length.Load()), - zap.Int64("size", e.TotalSize.Load())) - return errors.Trace(saveEngineMetaToDB(&e.localFileMeta, e.db)) -} - -func (e *File) loadEngineMeta() error { - jsonBytes, closer, err := e.db.Get(engineMetaKey) - if err != nil { - if err == pebble.ErrNotFound { - log.L().Debug("local db missing engine meta", zap.Stringer("uuid", e.UUID), zap.Error(err)) - return nil - } - return err - } - defer closer.Close() - - if err = json.Unmarshal(jsonBytes, &e.localFileMeta); err != nil { - log.L().Warn("local db failed to deserialize meta", zap.Stringer("uuid", e.UUID), zap.ByteString("content", jsonBytes), zap.Error(err)) - return err - } - log.L().Debug("load engine meta", zap.Stringer("uuid", e.UUID), zap.Int64("count", e.Length.Load()), - zap.Int64("size", e.TotalSize.Load())) - return nil -} - type local struct { - engines sync.Map // sync version of map[uuid.UUID]*File + engines sync.Map // sync version of map[uuid.UUID]*Engine pdCtl *pdutil.PdController conns common.GRPCConns @@ -828,64 +151,6 @@ type local struct { errorMgr *errormanager.ErrorManager } -// connPool is a lazy pool of gRPC channels. -// When `Get` called, it lazily allocates new connection if connection not full. -// If it's full, then it will return allocated channels round-robin. -type connPool struct { - mu sync.Mutex - - conns []*grpc.ClientConn - next int - cap int - newConn func(ctx context.Context) (*grpc.ClientConn, error) -} - -func (p *connPool) takeConns() (conns []*grpc.ClientConn) { - p.mu.Lock() - defer p.mu.Unlock() - p.conns, conns = nil, p.conns - p.next = 0 - return conns -} - -// Close closes the conn pool. -func (p *connPool) Close() { - for _, c := range p.takeConns() { - if err := c.Close(); err != nil { - log.L().Warn("failed to close clientConn", zap.String("target", c.Target()), log.ShortError(err)) - } - } -} - -// get tries to get an existing connection from the pool, or make a new one if the pool not full. -func (p *connPool) get(ctx context.Context) (*grpc.ClientConn, error) { - p.mu.Lock() - defer p.mu.Unlock() - if len(p.conns) < p.cap { - c, err := p.newConn(ctx) - if err != nil { - return nil, errors.Trace(err) - } - p.conns = append(p.conns, c) - return c, nil - } - - conn := p.conns[p.next] - p.next = (p.next + 1) % p.cap - return conn, nil -} - -// newConnPool creates a new connPool by the specified conn factory function and capacity. -func newConnPool(cap int, newConn func(ctx context.Context) (*grpc.ClientConn, error)) *connPool { - return &connPool{ - cap: cap, - conns: make([]*grpc.ClientConn, 0, cap), - newConn: newConn, - - mu: sync.Mutex{}, - } -} - var bufferPool = membuf.NewPool(1024, manual.Allocator{}) func openDuplicateDB(storeDir string) (*pebble.DB, error) { @@ -980,22 +245,22 @@ func NewLocalBackend( errorMgr: errorMgr, } local.conns = common.NewGRPCConns() - if err = local.checkMultiIngestSupport(ctx, pdCtl); err != nil { + if err = local.checkMultiIngestSupport(ctx); err != nil { return backend.MakeBackend(nil), err } return backend.MakeBackend(local), nil } -func (local *local) checkMultiIngestSupport(ctx context.Context, pdCtl *pdutil.PdController) error { - stores, err := pdCtl.GetPDClient().GetAllStores(ctx, pd.WithExcludeTombstone()) +func (local *local) checkMultiIngestSupport(ctx context.Context) error { + stores, err := local.pdCtl.GetPDClient().GetAllStores(ctx, pd.WithExcludeTombstone()) if err != nil { return errors.Trace(err) } hasTiFlash := false for _, s := range stores { - if version.IsTiFlash(s) { + if s.State == metapb.StoreState_Up && version.IsTiFlash(s) { hasTiFlash = true break } @@ -1052,31 +317,31 @@ func (local *local) checkMultiIngestSupport(ctx context.Context, pdCtl *pdutil.P return nil } -// rlock read locks a local file and returns the File instance if it exists. -func (local *local) rLockEngine(engineId uuid.UUID) *File { +// rlock read locks a local file and returns the Engine instance if it exists. +func (local *local) rLockEngine(engineId uuid.UUID) *Engine { if e, ok := local.engines.Load(engineId); ok { - engine := e.(*File) + engine := e.(*Engine) engine.rLock() return engine } return nil } -// lock locks a local file and returns the File instance if it exists. -func (local *local) lockEngine(engineID uuid.UUID, state importMutexState) *File { +// lock locks a local file and returns the Engine instance if it exists. +func (local *local) lockEngine(engineID uuid.UUID, state importMutexState) *Engine { if e, ok := local.engines.Load(engineID); ok { - engine := e.(*File) + engine := e.(*Engine) engine.lock(state) return engine } return nil } -// tryRLockAllEngines tries to read lock all engines, return all `File`s that are successfully locked. -func (local *local) tryRLockAllEngines() []*File { - var allEngines []*File +// tryRLockAllEngines tries to read lock all engines, return all `Engine`s that are successfully locked. +func (local *local) tryRLockAllEngines() []*Engine { + var allEngines []*Engine local.engines.Range(func(k, v interface{}) bool { - engine := v.(*File) + engine := v.(*Engine) // skip closed engine if engine.tryRLock() { if !engine.closed.Load() { @@ -1092,10 +357,10 @@ func (local *local) tryRLockAllEngines() []*File { // lockAllEnginesUnless tries to lock all engines, unless those which are already locked in the // state given by ignoreStateMask. Returns the list of locked engines. -func (local *local) lockAllEnginesUnless(newState, ignoreStateMask importMutexState) []*File { - var allEngines []*File +func (local *local) lockAllEnginesUnless(newState, ignoreStateMask importMutexState) []*Engine { + var allEngines []*Engine local.engines.Range(func(k, v interface{}) bool { - engine := v.(*File) + engine := v.(*Engine) if engine.lockUnless(newState, ignoreStateMask) { allEngines = append(allEngines, engine) } @@ -1200,17 +465,17 @@ func (local *local) Close() { // FlushEngine ensure the written data is saved successfully, to make sure no data lose after restart func (local *local) FlushEngine(ctx context.Context, engineID uuid.UUID) error { - engineFile := local.rLockEngine(engineID) + engine := local.rLockEngine(engineID) // the engine cannot be deleted after while we've acquired the lock identified by UUID. - if engineFile == nil { + if engine == nil { return errors.Errorf("engine '%s' not found", engineID) } - defer engineFile.rUnlock() - if engineFile.closed.Load() { + defer engine.rUnlock() + if engine.closed.Load() { return nil } - return engineFile.flushEngineWithoutLock(ctx) + return engine.flushEngineWithoutLock(ctx) } func (local *local) FlushAllEngines(parentCtx context.Context) (err error) { @@ -1222,10 +487,10 @@ func (local *local) FlushAllEngines(parentCtx context.Context) (err error) { }() eg, ctx := errgroup.WithContext(parentCtx) - for _, engineFile := range allEngines { - ef := engineFile + for _, engine := range allEngines { + e := engine eg.Go(func() error { - return ef.flushEngineWithoutLock(ctx) + return e.flushEngineWithoutLock(ctx) }) } return eg.Wait() @@ -1268,7 +533,7 @@ func (local *local) openEngineDB(engineUUID uuid.UUID, readOnly bool) (*pebble.D return db, errors.Trace(err) } -// This method must be called with holding mutex of File +// OpenEngine must be called with holding mutex of Engine. func (local *local) OpenEngine(ctx context.Context, cfg *backend.EngineConfig, engineUUID uuid.UUID) error { engineCfg := backend.LocalEngineConfig{} if cfg.Local != nil { @@ -1294,7 +559,7 @@ func (local *local) OpenEngine(ctx context.Context, cfg *backend.EngineConfig, e if local.duplicateDetection { keyAdapter = duplicateKeyAdapter{} } - e, _ := local.engines.LoadOrStore(engineUUID, &File{ + e, _ := local.engines.LoadOrStore(engineUUID, &Engine{ UUID: engineUUID, sstDir: sstDir, sstMetasChan: make(chan metaOrFlush, 64), @@ -1307,7 +572,7 @@ func (local *local) OpenEngine(ctx context.Context, cfg *backend.EngineConfig, e errorMgr: local.errorMgr, keyAdapter: keyAdapter, }) - engine := e.(*File) + engine := e.(*Engine) engine.db = db engine.sstIngester = dbSSTIngester{e: engine} if err = engine.loadEngineMeta(); err != nil { @@ -1321,7 +586,7 @@ func (local *local) OpenEngine(ctx context.Context, cfg *backend.EngineConfig, e return nil } -func (local *local) allocateTSIfNotExists(ctx context.Context, engine *File) error { +func (local *local) allocateTSIfNotExists(ctx context.Context, engine *Engine) error { if engine.TS > 0 { return nil } @@ -1338,14 +603,14 @@ func (local *local) allocateTSIfNotExists(ctx context.Context, engine *File) err func (local *local) CloseEngine(ctx context.Context, cfg *backend.EngineConfig, engineUUID uuid.UUID) error { // flush mem table to storage, to free memory, // ask others' advise, looks like unnecessary, but with this we can control memory precisely. - engine, ok := local.engines.Load(engineUUID) + engineI, ok := local.engines.Load(engineUUID) if !ok { // recovery mode, we should reopen this engine file db, err := local.openEngineDB(engineUUID, true) if err != nil { return err } - engineFile := &File{ + engine := &Engine{ UUID: engineUUID, db: db, sstMetasChan: make(chan metaOrFlush), @@ -1354,43 +619,47 @@ func (local *local) CloseEngine(ctx context.Context, cfg *backend.EngineConfig, duplicateDB: local.duplicateDB, errorMgr: local.errorMgr, } - engineFile.sstIngester = dbSSTIngester{e: engineFile} + engine.sstIngester = dbSSTIngester{e: engine} keyAdapter := KeyAdapter(noopKeyAdapter{}) if local.duplicateDetection { keyAdapter = duplicateKeyAdapter{} } - engineFile.keyAdapter = keyAdapter - if err = engineFile.loadEngineMeta(); err != nil { + engine.keyAdapter = keyAdapter + if err = engine.loadEngineMeta(); err != nil { return err } - local.engines.Store(engineUUID, engineFile) + local.engines.Store(engineUUID, engine) return nil } - engineFile := engine.(*File) - engineFile.rLock() - if engineFile.closed.Load() { - engineFile.rUnlock() + engine := engineI.(*Engine) + engine.rLock() + if engine.closed.Load() { + engine.rUnlock() return nil } - err := engineFile.flushEngineWithoutLock(ctx) - engineFile.rUnlock() + err := engine.flushEngineWithoutLock(ctx) + engine.rUnlock() // use mutex to make sure we won't close sstMetasChan while other routines // trying to do flush. - engineFile.lock(importMutexStateClose) - engineFile.closed.Store(true) - close(engineFile.sstMetasChan) - engineFile.unlock() + engine.lock(importMutexStateClose) + engine.closed.Store(true) + close(engine.sstMetasChan) + engine.unlock() if err != nil { return errors.Trace(err) } - engineFile.wg.Wait() - return engineFile.ingestErr.Get() + engine.wg.Wait() + return engine.ingestErr.Get() } func (local *local) getImportClient(ctx context.Context, storeID uint64) (sst.ImportSSTClient, error) { + return getImportClientFn(local, ctx, storeID) +} + +func getImportClient(local *local, ctx context.Context, storeID uint64) (sst.ImportSSTClient, error) { conn, err := local.getGrpcConn(ctx, storeID) if err != nil { return nil, err @@ -1408,7 +677,7 @@ type rangeStats struct { // tikv will takes the responsibility to do so. func (local *local) WriteToTiKV( ctx context.Context, - engineFile *File, + engine *Engine, region *split.RegionInfo, start, end []byte, regionSplitSize int64, @@ -1441,7 +710,7 @@ func (local *local) WriteToTiKV( begin := time.Now() regionRange := intersectRange(region.Region, Range{start: start, end: end}) opt := &pebble.IterOptions{LowerBound: regionRange.start, UpperBound: regionRange.end} - iter := newKeyIter(ctx, engineFile, opt) + iter := newKVIter(ctx, engine, opt) defer iter.Close() stats := rangeStats{} @@ -1499,7 +768,7 @@ func (local *local) WriteToTiKV( } req.Chunk = &sst.WriteRequest_Batch{ Batch: &sst.WriteBatch{ - CommitTs: engineFile.TS, + CommitTs: engine.TS, }, } clients = append(clients, wstream) @@ -1674,8 +943,8 @@ func splitRangeBySizeProps(fullRange Range, sizeProps *sizeProperties, sizeLimit return ranges } -func (local *local) readAndSplitIntoRange(ctx context.Context, engineFile *File, regionSplitSize int64, regionSplitKeys int64) ([]Range, error) { - iter := newKeyIter(ctx, engineFile, &pebble.IterOptions{}) +func (local *local) readAndSplitIntoRange(ctx context.Context, engine *Engine, regionSplitSize int64, regionSplitKeys int64) ([]Range, error) { + iter := newKVIter(ctx, engine, &pebble.IterOptions{}) defer iter.Close() iterError := func(e string) error { @@ -1699,8 +968,8 @@ func (local *local) readAndSplitIntoRange(ctx context.Context, engineFile *File, } endKey := nextKey(lastKey) - engineFileTotalSize := engineFile.TotalSize.Load() - engineFileLength := engineFile.Length.Load() + engineFileTotalSize := engine.TotalSize.Load() + engineFileLength := engine.Length.Load() // <= 96MB no need to split into range if engineFileTotalSize <= regionSplitSize && engineFileLength <= regionSplitKeys { @@ -1708,7 +977,7 @@ func (local *local) readAndSplitIntoRange(ctx context.Context, engineFile *File, return ranges, nil } - sizeProps, err := engineFile.getSizeProperties() + sizeProps, err := engine.getSizeProperties() if err != nil { return nil, errors.Trace(err) } @@ -1716,7 +985,7 @@ func (local *local) readAndSplitIntoRange(ctx context.Context, engineFile *File, ranges := splitRangeBySizeProps(Range{start: firstKey, end: endKey}, sizeProps, regionSplitSize, regionSplitKeys) - log.L().Info("split engine key ranges", zap.Stringer("engine", engineFile.UUID), + log.L().Info("split engine key ranges", zap.Stringer("engine", engine.UUID), zap.Int64("totalSize", engineFileTotalSize), zap.Int64("totalCount", engineFileLength), logutil.Key("firstKey", firstKey), logutil.Key("lastKey", lastKey), zap.Int("ranges", len(ranges))) @@ -1726,7 +995,7 @@ func (local *local) readAndSplitIntoRange(ctx context.Context, engineFile *File, func (local *local) writeAndIngestByRange( ctxt context.Context, - engineFile *File, + engine *Engine, start, end []byte, regionSplitSize int64, regionSplitKeys int64, @@ -1736,7 +1005,7 @@ func (local *local) writeAndIngestByRange( UpperBound: end, } - iter := newKeyIter(ctxt, engineFile, ito) + iter := newKVIter(ctxt, engine, ito) defer iter.Close() // Needs seek to first because NewIter returns an iterator that is unpositioned hasKey := iter.First() @@ -1747,7 +1016,7 @@ func (local *local) writeAndIngestByRange( log.L().Info("There is no pairs in iterator", logutil.Key("start", start), logutil.Key("end", end)) - engineFile.finishedRanges.add(Range{start: start, end: end}) + engine.finishedRanges.add(Range{start: start, end: end}) return nil } pairStart := append([]byte{}, iter.Key()...) @@ -1788,7 +1057,7 @@ WriteAndIngest: zap.Binary("end", region.Region.GetEndKey()), zap.Reflect("peers", region.Region.GetPeers())) w := local.ingestConcurrency.Apply() - err = local.writeAndIngestPairs(ctx, engineFile, region, pairStart, end, regionSplitSize, regionSplitKeys) + err = local.writeAndIngestPairs(ctx, engine, region, pairStart, end, regionSplitSize, regionSplitKeys) local.ingestConcurrency.Recycle(w) if err != nil { if common.IsContextCanceledError(err) { @@ -1823,7 +1092,7 @@ const ( func (local *local) writeAndIngestPairs( ctx context.Context, - engineFile *File, + engine *Engine, region *split.RegionInfo, start, end []byte, regionSplitSize int64, @@ -1836,7 +1105,7 @@ loopWrite: var metas []*sst.SSTMeta var finishedRange Range var rangeStats rangeStats - metas, finishedRange, rangeStats, err = local.WriteToTiKV(ctx, engineFile, region, start, end, regionSplitSize, regionSplitKeys) + metas, finishedRange, rangeStats, err = local.WriteToTiKV(ctx, engine, region, start, end, regionSplitSize, regionSplitKeys) if err != nil { if common.IsContextCanceledError(err) { return err @@ -1932,9 +1201,9 @@ loopWrite: logutil.Region(region.Region), logutil.Key("start", start), logutil.Key("end", end)) } else { - engineFile.importedKVSize.Add(rangeStats.totalBytes) - engineFile.importedKVCount.Add(rangeStats.count) - engineFile.finishedRanges.add(finishedRange) + engine.importedKVSize.Add(rangeStats.totalBytes) + engine.importedKVCount.Add(rangeStats.count) + engine.finishedRanges.add(finishedRange) metric.BytesCounter.WithLabelValues(metric.TableStateImported).Add(float64(rangeStats.totalBytes)) } return errors.Trace(err) @@ -1943,10 +1212,10 @@ loopWrite: return errors.Trace(err) } -func (local *local) writeAndIngestByRanges(ctx context.Context, engineFile *File, ranges []Range, regionSplitSize int64, regionSplitKeys int64) error { - if engineFile.Length.Load() == 0 { +func (local *local) writeAndIngestByRanges(ctx context.Context, engine *Engine, ranges []Range, regionSplitSize int64, regionSplitKeys int64) error { + if engine.Length.Load() == 0 { // engine is empty, this is likes because it's a index engine but the table contains no index - log.L().Info("engine contains no data", zap.Stringer("uuid", engineFile.UUID)) + log.L().Info("engine contains no data", zap.Stringer("uuid", engine.UUID)) return nil } log.L().Debug("the ranges Length write to tikv", zap.Int("Length", len(ranges))) @@ -1975,7 +1244,7 @@ func (local *local) writeAndIngestByRanges(ctx context.Context, engineFile *File // max retry backoff time: 2+4+8+16=30s backOffTime := time.Second for i := 0; i < maxRetryTimes; i++ { - err = local.writeAndIngestByRange(ctx, engineFile, startKey, endKey, regionSplitSize, regionSplitKeys) + err = local.writeAndIngestByRange(ctx, engine, startKey, endKey, regionSplitSize, regionSplitKeys) if err == nil || common.IsContextCanceledError(err) { return } @@ -2004,23 +1273,6 @@ func (local *local) writeAndIngestByRanges(ctx context.Context, engineFile *File return allErr } -type syncedRanges struct { - sync.Mutex - ranges []Range -} - -func (r *syncedRanges) add(g Range) { - r.Lock() - r.ranges = append(r.ranges, g) - r.Unlock() -} - -func (r *syncedRanges) reset() { - r.Lock() - r.ranges = r.ranges[:0] - r.Unlock() -} - func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regionSplitSize int64) error { lf := local.lockEngine(engineUUID, importMutexStateImport) if lf == nil { @@ -2232,72 +1484,6 @@ func (local *local) deleteDuplicateRows(ctx context.Context, logger *log.Task, h return nil } -func (e *File) unfinishedRanges(ranges []Range) []Range { - e.finishedRanges.Lock() - defer e.finishedRanges.Unlock() - - e.finishedRanges.ranges = sortAndMergeRanges(e.finishedRanges.ranges) - - return filterOverlapRange(ranges, e.finishedRanges.ranges) -} - -// sortAndMergeRanges sort the ranges and merge range that overlaps with each other into a single range. -func sortAndMergeRanges(ranges []Range) []Range { - if len(ranges) == 0 { - return ranges - } - - sort.Slice(ranges, func(i, j int) bool { - return bytes.Compare(ranges[i].start, ranges[j].start) < 0 - }) - - curEnd := ranges[0].end - i := 0 - for j := 1; j < len(ranges); j++ { - if bytes.Compare(curEnd, ranges[j].start) >= 0 { - if bytes.Compare(curEnd, ranges[j].end) < 0 { - curEnd = ranges[j].end - } - } else { - ranges[i].end = curEnd - i++ - ranges[i].start = ranges[j].start - curEnd = ranges[j].end - } - } - ranges[i].end = curEnd - return ranges[:i+1] -} - -func filterOverlapRange(ranges []Range, finishedRanges []Range) []Range { - if len(ranges) == 0 || len(finishedRanges) == 0 { - return ranges - } - - result := make([]Range, 0) - for _, r := range ranges { - start := r.start - end := r.end - for len(finishedRanges) > 0 && bytes.Compare(finishedRanges[0].start, end) < 0 { - fr := finishedRanges[0] - if bytes.Compare(fr.start, start) > 0 { - result = append(result, Range{start: start, end: fr.start}) - } - if bytes.Compare(fr.end, start) > 0 { - start = fr.end - } - if bytes.Compare(fr.end, end) > 0 { - break - } - finishedRanges = finishedRanges[1:] - } - if bytes.Compare(start, end) < 0 { - result = append(result, Range{start: start, end: end}) - } - } - return result -} - func (local *local) ResetEngine(ctx context.Context, engineUUID uuid.UUID) error { // the only way to reset the engine + reclaim the space is to delete and reopen it 🤷 localEngine := local.lockEngine(engineUUID, importMutexStateClose) @@ -2314,15 +1500,15 @@ func (local *local) ResetEngine(ctx context.Context, engineUUID uuid.UUID) error } db, err := local.openEngineDB(engineUUID, false) if err == nil { - // Reset localFileMeta except `Duplicates`. - meta := localFileMeta{ - Duplicates: *atomic.NewInt64(localEngine.localFileMeta.Duplicates.Load()), + // Reset engineMeta except `Duplicates`. + meta := engineMeta{ + Duplicates: *atomic.NewInt64(localEngine.engineMeta.Duplicates.Load()), } if err := saveEngineMetaToDB(&meta, db); err != nil { return errors.Trace(err) } localEngine.db = db - localEngine.localFileMeta = meta + localEngine.engineMeta = meta if !common.IsDirExists(localEngine.sstDir) { if err := os.Mkdir(localEngine.sstDir, 0o755); err != nil { return errors.Trace(err) @@ -2471,13 +1657,13 @@ func (local *local) LocalWriter(ctx context.Context, cfg *backend.LocalWriterCon if !ok { return nil, errors.Errorf("could not find engine for %s", engineUUID.String()) } - engineFile := e.(*File) - return openLocalWriter(cfg, engineFile, local.localWriterMemCacheSize) + engine := e.(*Engine) + return openLocalWriter(cfg, engine, local.localWriterMemCacheSize) } -func openLocalWriter(cfg *backend.LocalWriterConfig, f *File, cacheSize int64) (*Writer, error) { +func openLocalWriter(cfg *backend.LocalWriterConfig, engine *Engine, cacheSize int64) (*Writer, error) { w := &Writer{ - local: f, + engine: engine, memtableSizeLimit: cacheSize, kvBuffer: bufferPool.NewBuffer(), isKVSorted: cfg.IsKVSorted, @@ -2488,7 +1674,7 @@ func openLocalWriter(cfg *backend.LocalWriterConfig, f *File, cacheSize int64) ( if !w.isKVSorted { w.writeBatch = make([]common.KvPair, units.MiB) } - f.localWriters.Store(w, nil) + engine.localWriters.Store(w, nil) return w, nil } @@ -2608,684 +1794,11 @@ func nextKey(key []byte) []byte { return res } -type rangeOffsets struct { - Size uint64 - Keys uint64 -} - -type rangeProperty struct { - Key []byte - rangeOffsets -} - -func (r *rangeProperty) Less(than btree.Item) bool { - ta := than.(*rangeProperty) - return bytes.Compare(r.Key, ta.Key) < 0 -} - -var _ btree.Item = &rangeProperty{} - -type rangeProperties []rangeProperty - -func decodeRangeProperties(data []byte) (rangeProperties, error) { - r := make(rangeProperties, 0, 16) - for len(data) > 0 { - if len(data) < 4 { - return nil, io.ErrUnexpectedEOF - } - keyLen := int(binary.BigEndian.Uint32(data[:4])) - data = data[4:] - if len(data) < keyLen+8*2 { - return nil, io.ErrUnexpectedEOF - } - key := data[:keyLen] - data = data[keyLen:] - size := binary.BigEndian.Uint64(data[:8]) - keys := binary.BigEndian.Uint64(data[8:]) - data = data[16:] - r = append(r, rangeProperty{Key: key, rangeOffsets: rangeOffsets{Size: size, Keys: keys}}) - } - - return r, nil -} - -func (r rangeProperties) Encode() []byte { - b := make([]byte, 0, 1024) - idx := 0 - for _, p := range r { - b = append(b, 0, 0, 0, 0) - binary.BigEndian.PutUint32(b[idx:], uint32(len(p.Key))) - idx += 4 - b = append(b, p.Key...) - idx += len(p.Key) - - b = append(b, 0, 0, 0, 0, 0, 0, 0, 0) - binary.BigEndian.PutUint64(b[idx:], p.Size) - idx += 8 - - b = append(b, 0, 0, 0, 0, 0, 0, 0, 0) - binary.BigEndian.PutUint64(b[idx:], p.Keys) - idx += 8 - } - return b -} - -func (r rangeProperties) get(key []byte) rangeOffsets { - idx := sort.Search(len(r), func(i int) bool { - return bytes.Compare(r[i].Key, key) >= 0 - }) - return r[idx].rangeOffsets -} - -type RangePropertiesCollector struct { - props rangeProperties - lastOffsets rangeOffsets - lastKey []byte - currentOffsets rangeOffsets - propSizeIdxDistance uint64 - propKeysIdxDistance uint64 -} - -func newRangePropertiesCollector() pebble.TablePropertyCollector { - return &RangePropertiesCollector{ - props: make([]rangeProperty, 0, 1024), - propSizeIdxDistance: defaultPropSizeIndexDistance, - propKeysIdxDistance: defaultPropKeysIndexDistance, - } -} - -func (c *RangePropertiesCollector) sizeInLastRange() uint64 { - return c.currentOffsets.Size - c.lastOffsets.Size -} - -func (c *RangePropertiesCollector) keysInLastRange() uint64 { - return c.currentOffsets.Keys - c.lastOffsets.Keys -} - -func (c *RangePropertiesCollector) insertNewPoint(key []byte) { - c.lastOffsets = c.currentOffsets - c.props = append(c.props, rangeProperty{Key: append([]byte{}, key...), rangeOffsets: c.currentOffsets}) -} - -// implement `pebble.TablePropertyCollector` -// implement `TablePropertyCollector.Add` -func (c *RangePropertiesCollector) Add(key pebble.InternalKey, value []byte) error { - c.currentOffsets.Size += uint64(len(value)) + uint64(len(key.UserKey)) - c.currentOffsets.Keys++ - if len(c.lastKey) == 0 || c.sizeInLastRange() >= c.propSizeIdxDistance || - c.keysInLastRange() >= c.propKeysIdxDistance { - c.insertNewPoint(key.UserKey) - } - c.lastKey = append(c.lastKey[:0], key.UserKey...) - return nil -} - -func (c *RangePropertiesCollector) Finish(userProps map[string]string) error { - if c.sizeInLastRange() > 0 || c.keysInLastRange() > 0 { - c.insertNewPoint(c.lastKey) - } - - userProps[propRangeIndex] = string(c.props.Encode()) - return nil -} - -// The name of the property collector. -func (c *RangePropertiesCollector) Name() string { - return propRangeIndex -} - -type sizeProperties struct { - totalSize uint64 - indexHandles *btree.BTree -} - -func newSizeProperties() *sizeProperties { - return &sizeProperties{indexHandles: btree.New(32)} -} - -func (s *sizeProperties) add(item *rangeProperty) { - if old := s.indexHandles.ReplaceOrInsert(item); old != nil { - o := old.(*rangeProperty) - item.Keys += o.Keys - item.Size += o.Size - } -} - -func (s *sizeProperties) addAll(props rangeProperties) { - prevRange := rangeOffsets{} - for _, r := range props { - s.add(&rangeProperty{ - Key: r.Key, - rangeOffsets: rangeOffsets{Keys: r.Keys - prevRange.Keys, Size: r.Size - prevRange.Size}, - }) - prevRange = r.rangeOffsets - } - if len(props) > 0 { - s.totalSize = props[len(props)-1].Size - } -} - -// iter the tree until f return false -func (s *sizeProperties) iter(f func(p *rangeProperty) bool) { - s.indexHandles.Ascend(func(i btree.Item) bool { - prop := i.(*rangeProperty) - return f(prop) - }) -} - -type sstMeta struct { - path string - minKey []byte - maxKey []byte - totalSize int64 - totalCount int64 - // used for calculate disk-quota - fileSize int64 - seq int32 -} - -type Writer struct { - sync.Mutex - local *File - memtableSizeLimit int64 - - // if the KVs are append in order, we can directly write the into SST file, - // else we must first store them in writeBatch and then batch flush into SST file. - isKVSorted bool - writer *sstWriter - - // bytes buffer for writeBatch - kvBuffer *membuf.Buffer - writeBatch []common.KvPair - // if the kvs in writeBatch are in order, we can avoid doing a `sort.Slice` which - // is quite slow. in our bench, the sort operation eats about 5% of total CPU - isWriteBatchSorted bool - - batchCount int - batchSize int64 - totalSize int64 - totalCount int64 - - lastMetaSeq int32 -} - -func (w *Writer) appendRowsSorted(kvs []common.KvPair) error { - if w.writer == nil { - writer, err := w.createSSTWriter() - if err != nil { - return errors.Trace(err) - } - w.writer = writer - w.writer.minKey = append([]byte{}, kvs[0].Key...) - } - - totalKeyLen := 0 - for i := 0; i < len(kvs); i++ { - totalKeyLen += w.local.keyAdapter.EncodedLen(kvs[i].Key) - } - buf := make([]byte, totalKeyLen) - encodedKvs := make([]common.KvPair, len(kvs)) - for i := 0; i < len(kvs); i++ { - encodedKey := w.local.keyAdapter.Encode(buf, kvs[i].Key, kvs[i].RowID, kvs[i].Offset) - buf = buf[len(encodedKey):] - encodedKvs[i] = common.KvPair{Key: encodedKey, Val: kvs[i].Val} - w.batchSize += int64(len(encodedKvs[i].Key) + len(encodedKvs[i].Val)) - } - - w.batchCount += len(encodedKvs) - w.totalCount += int64(len(encodedKvs)) - return w.writer.writeKVs(encodedKvs) -} - -func (w *Writer) appendRowsUnsorted(ctx context.Context, kvs []common.KvPair) error { - l := len(w.writeBatch) - cnt := w.batchCount - var lastKey []byte - if cnt > 0 { - lastKey = w.writeBatch[cnt-1].Key - } - for _, pair := range kvs { - if w.isWriteBatchSorted && bytes.Compare(lastKey, pair.Key) > 0 { - w.isWriteBatchSorted = false - } - lastKey = pair.Key - w.batchSize += int64(len(pair.Key) + len(pair.Val)) - buf := w.kvBuffer.AllocBytes(w.local.keyAdapter.EncodedLen(pair.Key)) - key := w.local.keyAdapter.Encode(buf, pair.Key, pair.RowID, pair.Offset) - val := w.kvBuffer.AddBytes(pair.Val) - if cnt < l { - w.writeBatch[cnt].Key = key - w.writeBatch[cnt].Val = val - } else { - w.writeBatch = append(w.writeBatch, common.KvPair{Key: key, Val: val}) - } - cnt++ - } - w.batchCount = cnt - - if w.batchSize > w.memtableSizeLimit { - if err := w.flushKVs(ctx); err != nil { - return err - } - } - w.totalCount += int64(len(kvs)) - return nil -} - func (local *local) EngineFileSizes() (res []backend.EngineFileSize) { local.engines.Range(func(k, v interface{}) bool { - engine := v.(*File) + engine := v.(*Engine) res = append(res, engine.getEngineFileSize()) return true }) return } - -func (w *Writer) AppendRows(ctx context.Context, tableName string, columnNames []string, rows kv.Rows) error { - kvs := kv.KvPairsFromRows(rows) - if len(kvs) == 0 { - return nil - } - - if w.local.closed.Load() { - return errorEngineClosed - } - - w.Lock() - defer w.Unlock() - - // if chunk has _tidb_rowid field, we can't ensure that the rows are sorted. - if w.isKVSorted && w.writer == nil { - for _, c := range columnNames { - if c == model.ExtraHandleName.L { - w.isKVSorted = false - } - } - } - - if w.isKVSorted { - return w.appendRowsSorted(kvs) - } - return w.appendRowsUnsorted(ctx, kvs) -} - -func (w *Writer) flush(ctx context.Context) error { - w.Lock() - defer w.Unlock() - if w.batchCount == 0 { - return nil - } - - w.totalSize += w.batchSize - if len(w.writeBatch) > 0 { - if err := w.flushKVs(ctx); err != nil { - return errors.Trace(err) - } - } - - if w.writer != nil { - meta, err := w.writer.close() - if err != nil { - return errors.Trace(err) - } - w.writer = nil - w.batchCount = 0 - if meta != nil && meta.totalSize > 0 { - return w.addSST(ctx, meta) - } - } - - return nil -} - -type flushStatus struct { - local *File - seq int32 -} - -func (f flushStatus) Flushed() bool { - return f.seq <= f.local.finishedMetaSeq.Load() -} - -func (w *Writer) Close(ctx context.Context) (backend.ChunkFlushStatus, error) { - defer w.kvBuffer.Destroy() - defer w.local.localWriters.Delete(w) - err := w.flush(ctx) - // FIXME: in theory this line is useless, but In our benchmark with go1.15 - // this can resolve the memory consistently increasing issue. - // maybe this is a bug related to go GC mechanism. - w.writeBatch = nil - return flushStatus{local: w.local, seq: w.lastMetaSeq}, err -} - -func (w *Writer) IsSynced() bool { - return w.batchCount == 0 && w.lastMetaSeq <= w.local.finishedMetaSeq.Load() -} - -func (w *Writer) flushKVs(ctx context.Context) error { - writer, err := w.createSSTWriter() - if err != nil { - return errors.Trace(err) - } - if !w.isWriteBatchSorted { - sort.Slice(w.writeBatch[:w.batchCount], func(i, j int) bool { - return bytes.Compare(w.writeBatch[i].Key, w.writeBatch[j].Key) < 0 - }) - w.isWriteBatchSorted = true - } - - writer.minKey = append(writer.minKey[:0], w.writeBatch[0].Key...) - err = writer.writeKVs(w.writeBatch[:w.batchCount]) - if err != nil { - return errors.Trace(err) - } - meta, err := writer.close() - if err != nil { - return errors.Trace(err) - } - err = w.addSST(ctx, meta) - if err != nil { - return errors.Trace(err) - } - - w.totalSize += w.batchSize - w.batchSize = 0 - w.batchCount = 0 - w.kvBuffer.Reset() - return nil -} - -func (w *Writer) addSST(ctx context.Context, meta *sstMeta) error { - seq, err := w.local.addSST(ctx, meta) - if err != nil { - return err - } - w.lastMetaSeq = seq - return nil -} - -func (w *Writer) createSSTWriter() (*sstWriter, error) { - path := filepath.Join(w.local.sstDir, uuid.New().String()+".sst") - writer, err := newSSTWriter(path) - if err != nil { - return nil, err - } - sw := &sstWriter{sstMeta: &sstMeta{path: path}, writer: writer} - return sw, nil -} - -var errorUnorderedSSTInsertion = errors.New("inserting KVs into SST without order") - -type sstWriter struct { - *sstMeta - writer *sstable.Writer -} - -func newSSTWriter(path string) (*sstable.Writer, error) { - f, err := os.Create(path) - if err != nil { - return nil, errors.Trace(err) - } - writer := sstable.NewWriter(f, sstable.WriterOptions{ - TablePropertyCollectors: []func() pebble.TablePropertyCollector{ - newRangePropertiesCollector, - }, - BlockSize: 16 * 1024, - }) - return writer, nil -} - -func (sw *sstWriter) writeKVs(kvs []common.KvPair) error { - if len(kvs) == 0 { - return nil - } - - if bytes.Compare(kvs[0].Key, sw.maxKey) <= 0 { - return errorUnorderedSSTInsertion - } - - internalKey := sstable.InternalKey{ - Trailer: uint64(sstable.InternalKeyKindSet), - } - var lastKey []byte - for _, p := range kvs { - if bytes.Equal(p.Key, lastKey) { - log.L().Warn("duplicated key found, skip write", logutil.Key("key", p.Key)) - continue - } - internalKey.UserKey = p.Key - if err := sw.writer.Add(internalKey, p.Val); err != nil { - return errors.Trace(err) - } - sw.totalSize += int64(len(p.Key)) + int64(len(p.Val)) - } - sw.totalCount += int64(len(kvs)) - sw.maxKey = append(sw.maxKey[:0], kvs[len(kvs)-1].Key...) - return nil -} - -func (sw *sstWriter) close() (*sstMeta, error) { - if err := sw.writer.Close(); err != nil { - return nil, errors.Trace(err) - } - meta, err := sw.writer.Metadata() - if err != nil { - return nil, errors.Trace(err) - } - sw.fileSize = int64(meta.Size) - return sw.sstMeta, nil -} - -type sstIter struct { - name string - key []byte - val []byte - iter sstable.Iterator - reader *sstable.Reader - valid bool -} - -func (i *sstIter) Close() error { - if err := i.iter.Close(); err != nil { - return errors.Trace(err) - } - err := i.reader.Close() - return errors.Trace(err) -} - -type sstIterHeap struct { - iters []*sstIter -} - -func (h *sstIterHeap) Len() int { - return len(h.iters) -} - -func (h *sstIterHeap) Less(i, j int) bool { - return bytes.Compare(h.iters[i].key, h.iters[j].key) < 0 -} - -func (h *sstIterHeap) Swap(i, j int) { - h.iters[i], h.iters[j] = h.iters[j], h.iters[i] -} - -func (h *sstIterHeap) Push(x interface{}) { - h.iters = append(h.iters, x.(*sstIter)) -} - -func (h *sstIterHeap) Pop() interface{} { - item := h.iters[len(h.iters)-1] - h.iters = h.iters[:len(h.iters)-1] - return item -} - -func (h *sstIterHeap) Next() ([]byte, []byte, error) { - for { - if len(h.iters) == 0 { - return nil, nil, nil - } - - iter := h.iters[0] - if iter.valid { - iter.valid = false - return iter.key, iter.val, iter.iter.Error() - } - - var k *pebble.InternalKey - k, iter.val = iter.iter.Next() - if k != nil { - iter.key = k.UserKey - iter.valid = true - heap.Fix(h, 0) - } else { - err := iter.Close() - heap.Remove(h, 0) - if err != nil { - return nil, nil, errors.Trace(err) - } - } - } -} - -// sstIngester is a interface used to merge and ingest SST files. -// it's a interface mainly used for test convenience -type sstIngester interface { - mergeSSTs(metas []*sstMeta, dir string) (*sstMeta, error) - ingest([]*sstMeta) error -} - -type dbSSTIngester struct { - e *File -} - -func (i dbSSTIngester) mergeSSTs(metas []*sstMeta, dir string) (*sstMeta, error) { - if len(metas) == 0 { - return nil, errors.New("sst metas is empty") - } else if len(metas) == 1 { - return metas[0], nil - } - - start := time.Now() - newMeta := &sstMeta{ - seq: metas[len(metas)-1].seq, - } - mergeIter := &sstIterHeap{ - iters: make([]*sstIter, 0, len(metas)), - } - - for _, p := range metas { - f, err := os.Open(p.path) - if err != nil { - return nil, errors.Trace(err) - } - reader, err := sstable.NewReader(f, sstable.ReaderOptions{}) - if err != nil { - return nil, errors.Trace(err) - } - iter, err := reader.NewIter(nil, nil) - if err != nil { - return nil, errors.Trace(err) - } - key, val := iter.Next() - if key == nil { - continue - } - if iter.Error() != nil { - return nil, errors.Trace(iter.Error()) - } - mergeIter.iters = append(mergeIter.iters, &sstIter{ - name: p.path, - iter: iter, - key: key.UserKey, - val: val, - reader: reader, - valid: true, - }) - newMeta.totalSize += p.totalSize - newMeta.totalCount += p.totalCount - } - heap.Init(mergeIter) - - name := filepath.Join(dir, fmt.Sprintf("%s.sst", uuid.New())) - writer, err := newSSTWriter(name) - if err != nil { - return nil, errors.Trace(err) - } - newMeta.path = name - - internalKey := sstable.InternalKey{ - Trailer: uint64(sstable.InternalKeyKindSet), - } - key, val, err := mergeIter.Next() - if err != nil { - return nil, err - } - if key == nil { - return nil, errors.New("all ssts are empty!") - } - newMeta.minKey = append(newMeta.minKey[:0], key...) - lastKey := make([]byte, 0) - for { - if bytes.Equal(lastKey, key) { - log.L().Warn("duplicated key found, skipped", zap.Binary("key", lastKey)) - newMeta.totalCount-- - newMeta.totalSize -= int64(len(key) + len(val)) - - goto nextKey - } - internalKey.UserKey = key - err = writer.Add(internalKey, val) - if err != nil { - return nil, err - } - lastKey = append(lastKey[:0], key...) - nextKey: - key, val, err = mergeIter.Next() - if err != nil { - return nil, err - } - if key == nil { - break - } - } - err = writer.Close() - if err != nil { - return nil, errors.Trace(err) - } - meta, err := writer.Metadata() - if err != nil { - return nil, errors.Trace(err) - } - newMeta.maxKey = lastKey - newMeta.fileSize = int64(meta.Size) - - dur := time.Since(start) - log.L().Info("compact sst", zap.Int("fileCount", len(metas)), zap.Int64("size", newMeta.totalSize), - zap.Int64("count", newMeta.totalCount), zap.Duration("cost", dur), zap.String("file", name)) - - // async clean raw SSTs. - go func() { - totalSize := int64(0) - for _, m := range metas { - totalSize += m.fileSize - if err := os.Remove(m.path); err != nil { - log.L().Warn("async cleanup sst file failed", zap.Error(err)) - } - } - // decrease the pending size after clean up - i.e.pendingFileSize.Sub(totalSize) - }() - - return newMeta, err -} - -func (i dbSSTIngester) ingest(metas []*sstMeta) error { - if len(metas) == 0 { - return nil - } - paths := make([]string, 0, len(metas)) - for _, m := range metas { - paths = append(paths, m.path) - } - return i.e.db.Ingest(paths) -} diff --git a/br/pkg/lightning/backend/local/local_test.go b/br/pkg/lightning/backend/local/local_test.go index d1f700210ea2a..747034068c463 100644 --- a/br/pkg/lightning/backend/local/local_test.go +++ b/br/pkg/lightning/backend/local/local_test.go @@ -37,12 +37,19 @@ import ( "github.com/pingcap/kvproto/pkg/errorpb" sst "github.com/pingcap/kvproto/pkg/import_sstpb" "github.com/pingcap/kvproto/pkg/metapb" + pd "github.com/tikv/pd/client" + "google.golang.org/grpc" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" + "github.com/pingcap/tidb/br/pkg/lightning/backend" "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/br/pkg/mock" + "github.com/pingcap/tidb/br/pkg/pdutil" "github.com/pingcap/tidb/br/pkg/restore" + "github.com/pingcap/tidb/br/pkg/version" tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/tablecodec" @@ -54,6 +61,7 @@ import ( type localSuite struct{} var _ = Suite(&localSuite{}) +var _ = SerialSuites(&testMultiIngestSuite{}) func Test(t *testing.T) { TestingT(t) @@ -336,7 +344,7 @@ func testLocalWriter(c *C, needSort bool, partitialSort bool) { _, engineUUID := backend.MakeUUID("ww", 0) engineCtx, cancel := context.WithCancel(context.Background()) - f := &File{ + f := &Engine{ db: db, UUID: engineUUID, sstDir: tmpPath, @@ -565,7 +573,7 @@ func (s *localSuite) TestLocalIngestLoop(c *C) { c.Assert(err, IsNil) _, engineUUID := backend.MakeUUID("ww", 0) engineCtx, cancel := context.WithCancel(context.Background()) - f := File{ + f := Engine{ db: db, UUID: engineUUID, sstDir: "", @@ -782,7 +790,7 @@ func (s *localSuite) testMergeSSTs(c *C, kvs [][]common.KvPair, meta *sstMeta) { _, engineUUID := backend.MakeUUID("ww", 0) engineCtx, cancel := context.WithCancel(context.Background()) - f := &File{ + f := &Engine{ db: db, UUID: engineUUID, sstDir: tmpPath, @@ -866,3 +874,343 @@ func (s *localSuite) TestMergeSSTsDuplicated(c *C) { s.testMergeSSTs(c, kvs, &sstMeta{totalCount: 40, totalSize: 640}) } + +type mockPdClient struct { + pd.Client + stores []*metapb.Store +} + +func (c *mockPdClient) GetAllStores(ctx context.Context, opts ...pd.GetStoreOption) ([]*metapb.Store, error) { + return c.stores, nil +} + +type mockGrpcErr struct{} + +func (e mockGrpcErr) GRPCStatus() *status.Status { + return status.New(codes.Unimplemented, "unimplmented") +} + +func (e mockGrpcErr) Error() string { + return "unimplmented" +} + +type mockImportClient struct { + sst.ImportSSTClient + stores []*metapb.Store + curStore *metapb.Store + err error + retry int + cnt int + multiIngestCheckFn func(s *metapb.Store) bool +} + +func (c *mockImportClient) MultiIngest(context.Context, *sst.MultiIngestRequest, ...grpc.CallOption) (*sst.IngestResponse, error) { + defer func() { + c.cnt++ + }() + if c.cnt < c.retry && c.err != nil { + return nil, c.err + } + + if !c.multiIngestCheckFn(c.curStore) { + return nil, mockGrpcErr{} + } + return nil, nil +} + +type testMultiIngestSuite struct { + local *local + pdCli *mockPdClient +} + +func (s *testMultiIngestSuite) SetUpSuite(c *C) { + local := &local{ + pdCtl: &pdutil.PdController{}, + } + pdCli := &mockPdClient{} + local.pdCtl.SetPDClient(pdCli) + s.local = local + s.pdCli = pdCli +} + +func (s *testMultiIngestSuite) TestMultiIngest(c *C) { + defer func() { + getImportClientFn = getImportClient + }() + + allStores := []*metapb.Store{ + { + Id: 1, + State: metapb.StoreState_Offline, + }, + { + Id: 2, + State: metapb.StoreState_Tombstone, + Labels: []*metapb.StoreLabel{ + { + Key: "test", + Value: "tiflash", + }, + }, + }, + { + Id: 3, + State: metapb.StoreState_Up, + Labels: []*metapb.StoreLabel{ + { + Key: "test", + Value: "123", + }, + }, + }, + { + Id: 4, + State: metapb.StoreState_Tombstone, + Labels: []*metapb.StoreLabel{ + { + Key: "engine", + Value: "test", + }, + }, + }, + { + Id: 5, + State: metapb.StoreState_Tombstone, + Labels: []*metapb.StoreLabel{ + { + Key: "engine", + Value: "test123", + }, + }, + }, + { + Id: 6, + State: metapb.StoreState_Offline, + Labels: []*metapb.StoreLabel{ + { + Key: "engine", + Value: "tiflash", + }, + }, + }, + { + Id: 7, + State: metapb.StoreState_Up, + Labels: []*metapb.StoreLabel{ + { + Key: "test", + Value: "123", + }, + { + Key: "engine", + Value: "tiflash", + }, + }, + }, + { + Id: 8, + State: metapb.StoreState_Up, + }, + } + cases := []struct { + filter func(store *metapb.Store) bool + multiIngestSupport func(s *metapb.Store) bool + retry int + err error + supportMutliIngest bool + retErr string + }{ + // test up stores with all support multiIngest + { + func(store *metapb.Store) bool { + return store.State == metapb.StoreState_Up + }, + func(s *metapb.Store) bool { + return true + }, + 0, + nil, + true, + "", + }, + // test all up stores with tiflash not support multi ingest + { + func(store *metapb.Store) bool { + return store.State == metapb.StoreState_Up + }, + func(s *metapb.Store) bool { + return !version.IsTiFlash(s) + }, + 0, + nil, + true, + "", + }, + // test all up stores with only tiflash support multi ingest + { + func(store *metapb.Store) bool { + return store.State == metapb.StoreState_Up + }, + func(s *metapb.Store) bool { + return version.IsTiFlash(s) + }, + 0, + nil, + false, + "", + }, + // test all up stores with some non-tiflash store support multi ingest + { + func(store *metapb.Store) bool { + return store.State == metapb.StoreState_Up + }, + func(s *metapb.Store) bool { + return len(s.Labels) > 0 + }, + 0, + nil, + false, + "", + }, + // test all stores with all states + { + func(store *metapb.Store) bool { + return true + }, + func(s *metapb.Store) bool { + return true + }, + 0, + nil, + true, + "", + }, + // test all non-tiflash stores that support multi ingests + { + func(store *metapb.Store) bool { + return !version.IsTiFlash(store) + }, + func(s *metapb.Store) bool { + return !version.IsTiFlash(s) + }, + 0, + nil, + true, + "", + }, + // test only up stores support multi ingest + { + func(store *metapb.Store) bool { + return true + }, + func(s *metapb.Store) bool { + return s.State == metapb.StoreState_Up + }, + 0, + nil, + true, + "", + }, + // test only offline/tombstore stores support multi ingest + { + func(store *metapb.Store) bool { + return true + }, + func(s *metapb.Store) bool { + return s.State != metapb.StoreState_Up + }, + 0, + nil, + false, + "", + }, + // test grpc return error but no tiflash + { + func(store *metapb.Store) bool { + return !version.IsTiFlash(store) + }, + func(s *metapb.Store) bool { + return true + }, + math.MaxInt32, + errors.New("mock error"), + false, + "", + }, + // test grpc return error and contains offline tiflash + { + func(store *metapb.Store) bool { + return !version.IsTiFlash(store) || store.State != metapb.StoreState_Up + }, + func(s *metapb.Store) bool { + return true + }, + math.MaxInt32, + errors.New("mock error"), + false, + "", + }, + // test grpc return error + { + func(store *metapb.Store) bool { + return true + }, + func(s *metapb.Store) bool { + return true + }, + math.MaxInt32, + errors.New("mock error"), + false, + "mock error", + }, + // test grpc return error only once + { + func(store *metapb.Store) bool { + return true + }, + func(s *metapb.Store) bool { + return true + }, + 1, + errors.New("mock error"), + true, + "", + }, + } + + for _, testCase := range cases { + stores := make([]*metapb.Store, 0, len(allStores)) + for _, s := range allStores { + if testCase.filter(s) { + stores = append(stores, s) + } + } + + importCli := &mockImportClient{ + stores: allStores, + cnt: 0, + retry: testCase.retry, + err: testCase.err, + multiIngestCheckFn: testCase.multiIngestSupport, + } + s.pdCli.stores = stores + + getImportClientFn = func(local *local, ctx context.Context, storeID uint64) (sst.ImportSSTClient, error) { + for _, store := range importCli.stores { + if store.Id == storeID { + importCli.curStore = store + break + } + } + return importCli, nil + } + s.local.supportMultiIngest = false + + err := s.local.checkMultiIngestSupport(context.Background()) + if err != nil { + c.Assert(err, ErrorMatches, testCase.retErr) + } else { + c.Assert(s.local.supportMultiIngest, Equals, testCase.supportMutliIngest) + } + } +} diff --git a/br/pkg/lightning/backend/tidb/tidb.go b/br/pkg/lightning/backend/tidb/tidb.go index 1b95fe558ef88..c748da0c76625 100644 --- a/br/pkg/lightning/backend/tidb/tidb.go +++ b/br/pkg/lightning/backend/tidb/tidb.go @@ -84,6 +84,9 @@ type tidbEncoder struct { // the index of table columns for each data field. // index == len(table.columns) means this field is `_tidb_rowid` columnIdx []int + // the max index used in this chunk, due to the ignore-columns config, we can't + // directly check the total column count, so we fall back to only check that + // the there are enough columns. columnCnt int } @@ -284,22 +287,27 @@ func (enc *tidbEncoder) Encode(logger log.Logger, row []types.Datum, _ int64, co cols := enc.tbl.Cols() if len(enc.columnIdx) == 0 { - columnCount := 0 + columnMaxIdx := -1 columnIdx := make([]int, len(columnPermutation)) + for i := 0; i < len(columnPermutation); i++ { + columnIdx[i] = -1 + } for i, idx := range columnPermutation { if idx >= 0 { columnIdx[idx] = i - columnCount++ + if idx > columnMaxIdx { + columnMaxIdx = idx + } } } enc.columnIdx = columnIdx - enc.columnCnt = columnCount + enc.columnCnt = columnMaxIdx + 1 } // TODO: since the column count doesn't exactly reflect the real column names, we only check the upper bound currently. // See: tests/generated_columns/data/gencol.various_types.0.sql this sql has no columns, so encodeLoop will fill the // column permutation with default, thus enc.columnCnt > len(row). - if len(row) > enc.columnCnt { + if len(row) < enc.columnCnt { logger.Error("column count mismatch", zap.Ints("column_permutation", columnPermutation), zap.Array("data", kv.RowArrayMarshaler(row))) return emptyTiDBRow, errors.Errorf("column count mismatch, expected %d, got %d", enc.columnCnt, len(row)) @@ -308,8 +316,12 @@ func (enc *tidbEncoder) Encode(logger log.Logger, row []types.Datum, _ int64, co var encoded strings.Builder encoded.Grow(8 * len(row)) encoded.WriteByte('(') + cnt := 0 for i, field := range row { - if i != 0 { + if enc.columnIdx[i] < 0 { + continue + } + if cnt > 0 { encoded.WriteByte(',') } datum := field @@ -321,6 +333,7 @@ func (enc *tidbEncoder) Encode(logger log.Logger, row []types.Datum, _ int64, co ) return nil, err } + cnt++ } encoded.WriteByte(')') return tidbRow{ @@ -569,7 +582,7 @@ func (be *tidbBackend) FetchRemoteTableModels(ctx context.Context, schemaName st serverInfo := version.ParseServerInfo(versionStr) rows, e := tx.Query(` - SELECT table_name, column_name, column_type, extra + SELECT table_name, column_name, column_type, generation_expression, extra FROM information_schema.columns WHERE table_schema = ? ORDER BY table_name, ordinal_position; @@ -585,8 +598,8 @@ func (be *tidbBackend) FetchRemoteTableModels(ctx context.Context, schemaName st curTable *model.TableInfo ) for rows.Next() { - var tableName, columnName, columnType, columnExtra string - if e := rows.Scan(&tableName, &columnName, &columnType, &columnExtra); e != nil { + var tableName, columnName, columnType, generationExpr, columnExtra string + if e := rows.Scan(&tableName, &columnName, &columnType, &generationExpr, &columnExtra); e != nil { return e } if tableName != curTableName { @@ -615,6 +628,7 @@ func (be *tidbBackend) FetchRemoteTableModels(ctx context.Context, schemaName st FieldType: types.FieldType{ Flag: flag, }, + GeneratedExprString: generationExpr, }) curColOffset++ } diff --git a/br/pkg/lightning/backend/tidb/tidb_test.go b/br/pkg/lightning/backend/tidb/tidb_test.go index e37f53d12b27e..0e01395306c39 100644 --- a/br/pkg/lightning/backend/tidb/tidb_test.go +++ b/br/pkg/lightning/backend/tidb/tidb_test.go @@ -22,7 +22,6 @@ import ( "testing" "github.com/DATA-DOG/go-sqlmock" - . "github.com/pingcap/check" "github.com/pingcap/tidb/br/pkg/lightning/backend" "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" "github.com/pingcap/tidb/br/pkg/lightning/backend/tidb" @@ -36,15 +35,10 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" + "github.com/stretchr/testify/require" "go.uber.org/atomic" ) -func Test(t *testing.T) { - TestingT(t) -} - -var _ = Suite(&mysqlSuite{}) - type mysqlSuite struct { dbHandle *sql.DB mockDB sqlmock.Sqlmock @@ -52,9 +46,9 @@ type mysqlSuite struct { tbl table.Table } -func (s *mysqlSuite) SetUpTest(c *C) { +func createMysqlSuite(t *testing.T) *mysqlSuite { db, mock, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) tys := []byte{ mysql.TypeLong, mysql.TypeLong, mysql.TypeTiny, mysql.TypeInt24, mysql.TypeFloat, mysql.TypeDouble, @@ -67,29 +61,29 @@ func (s *mysqlSuite) SetUpTest(c *C) { } tblInfo := &model.TableInfo{ID: 1, Columns: cols, PKIsHandle: false, State: model.StatePublic} tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(0), tblInfo) - c.Assert(err, IsNil) - - s.dbHandle = db - s.mockDB = mock - s.backend = tidb.NewTiDBBackend(db, config.ReplaceOnDup, errormanager.New(nil, config.NewConfig())) - s.tbl = tbl + require.NoError(t, err) + backend := tidb.NewTiDBBackend(db, config.ReplaceOnDup, errormanager.New(nil, config.NewConfig())) + return &mysqlSuite{dbHandle: db, mockDB: mock, backend: backend, tbl: tbl} } -func (s *mysqlSuite) TearDownTest(c *C) { +func (s *mysqlSuite) TearDownTest(t *testing.T) { s.backend.Close() - c.Assert(s.mockDB.ExpectationsWereMet(), IsNil) + require.NoError(t, s.mockDB.ExpectationsWereMet()) } -func (s *mysqlSuite) TestWriteRowsReplaceOnDup(c *C) { +func TestWriteRowsReplaceOnDup(t *testing.T) { + t.Parallel() + s := createMysqlSuite(t) + defer s.TearDownTest(t) s.mockDB. - ExpectExec("\\QREPLACE INTO `foo`.`bar`(`a`,`b`,`c`,`d`,`e`,`f`,`g`,`h`,`i`,`j`,`k`,`l`,`m`,`n`,`o`) VALUES(18446744073709551615,-9223372036854775808,0,NULL,7.5,5e-324,1.7976931348623157e+308,0,'甲乙丙\\r\\n\\0\\Z''\"\\\\`',x'000000abcdef',2557891634,'12.5',51)\\E"). + ExpectExec("\\QREPLACE INTO `foo`.`bar`(`b`,`d`,`e`,`f`,`g`,`h`,`i`,`j`,`k`,`l`,`m`,`n`,`o`) VALUES(-9223372036854775808,NULL,7.5,5e-324,1.7976931348623157e+308,0,'甲乙丙\\r\\n\\0\\Z''\"\\\\`',x'000000abcdef',2557891634,'12.5',51)\\E"). WillReturnResult(sqlmock.NewResult(1, 1)) ctx := context.Background() logger := log.L() engine, err := s.backend.OpenEngine(ctx, &backend.EngineConfig{}, "`foo`.`bar`", 1) - c.Assert(err, IsNil) + require.NoError(t, err) dataRows := s.backend.MakeEmptyRows() dataChecksum := verification.MakeKVChecksum(0, 0, 0) @@ -102,8 +96,11 @@ func (s *mysqlSuite) TestWriteRowsReplaceOnDup(c *C) { perms = append(perms, i) } perms = append(perms, -1) + // skip column a,c due to ignore-columns + perms[0] = -1 + perms[2] = -1 encoder, err := s.backend.NewEncoder(s.tbl, &kv.SessionOptions{SQLMode: 0, Timestamp: 1234567890}) - c.Assert(err, IsNil) + require.NoError(t, err) row, err := encoder.Encode(logger, []types.Datum{ types.NewUintDatum(18446744073709551615), types.NewIntDatum(-9223372036854775808), @@ -121,19 +118,22 @@ func (s *mysqlSuite) TestWriteRowsReplaceOnDup(c *C) { types.NewDecimalDatum(types.NewDecFromFloatForTest(12.5)), types.NewMysqlEnumDatum(types.Enum{Name: "ENUM_NAME", Value: 51}), }, 1, perms, "0.csv", 0) - c.Assert(err, IsNil) + require.NoError(t, err) row.ClassifyAndAppend(&dataRows, &dataChecksum, &indexRows, &indexChecksum) writer, err := engine.LocalWriter(ctx, nil) - c.Assert(err, IsNil) - err = writer.WriteRows(ctx, []string{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l", "m", "n", "o"}, dataRows) - c.Assert(err, IsNil) + require.NoError(t, err) + err = writer.WriteRows(ctx, []string{"b", "d", "e", "f", "g", "h", "i", "j", "k", "l", "m", "n", "o"}, dataRows) + require.NoError(t, err) st, err := writer.Close(ctx) - c.Assert(err, IsNil) - c.Assert(st, IsNil) + require.NoError(t, err) + require.Nil(t, st) } -func (s *mysqlSuite) TestWriteRowsIgnoreOnDup(c *C) { +func TestWriteRowsIgnoreOnDup(t *testing.T) { + t.Parallel() + s := createMysqlSuite(t) + defer s.TearDownTest(t) s.mockDB. ExpectExec("\\QINSERT IGNORE INTO `foo`.`bar`(`a`) VALUES(1)\\E"). WillReturnResult(sqlmock.NewResult(1, 1)) @@ -143,7 +143,7 @@ func (s *mysqlSuite) TestWriteRowsIgnoreOnDup(c *C) { ignoreBackend := tidb.NewTiDBBackend(s.dbHandle, config.IgnoreOnDup, errormanager.New(nil, config.NewConfig())) engine, err := ignoreBackend.OpenEngine(ctx, &backend.EngineConfig{}, "`foo`.`bar`", 1) - c.Assert(err, IsNil) + require.NoError(t, err) dataRows := ignoreBackend.MakeEmptyRows() dataChecksum := verification.MakeKVChecksum(0, 0, 0) @@ -151,33 +151,36 @@ func (s *mysqlSuite) TestWriteRowsIgnoreOnDup(c *C) { indexChecksum := verification.MakeKVChecksum(0, 0, 0) encoder, err := ignoreBackend.NewEncoder(s.tbl, &kv.SessionOptions{}) - c.Assert(err, IsNil) + require.NoError(t, err) row, err := encoder.Encode(logger, []types.Datum{ types.NewIntDatum(1), - }, 1, []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, -1}, "1.csv", 0) - c.Assert(err, IsNil) + }, 1, []int{0, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1}, "1.csv", 0) + require.NoError(t, err) row.ClassifyAndAppend(&dataRows, &dataChecksum, &indexRows, &indexChecksum) writer, err := engine.LocalWriter(ctx, nil) - c.Assert(err, IsNil) + require.NoError(t, err) err = writer.WriteRows(ctx, []string{"a"}, dataRows) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = writer.Close(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) // test encode rows with _tidb_rowid encoder, err = ignoreBackend.NewEncoder(s.tbl, &kv.SessionOptions{}) - c.Assert(err, IsNil) + require.NoError(t, err) rowWithID, err := encoder.Encode(logger, []types.Datum{ types.NewIntDatum(1), types.NewIntDatum(1), // _tidb_rowid field }, 1, []int{0, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, 1}, "2.csv", 0) - c.Assert(err, IsNil) + require.NoError(t, err) // tidbRow is stringer. - c.Assert(fmt.Sprint(rowWithID), Equals, "(1,1)") + require.Equal(t, "(1,1)", fmt.Sprint(rowWithID)) } -func (s *mysqlSuite) TestWriteRowsErrorOnDup(c *C) { +func TestWriteRowsErrorOnDup(t *testing.T) { + t.Parallel() + s := createMysqlSuite(t) + defer s.TearDownTest(t) s.mockDB. ExpectExec("\\QINSERT INTO `foo`.`bar`(`a`) VALUES(1)\\E"). WillReturnResult(sqlmock.NewResult(1, 1)) @@ -187,7 +190,7 @@ func (s *mysqlSuite) TestWriteRowsErrorOnDup(c *C) { ignoreBackend := tidb.NewTiDBBackend(s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig())) engine, err := ignoreBackend.OpenEngine(ctx, &backend.EngineConfig{}, "`foo`.`bar`", 1) - c.Assert(err, IsNil) + require.NoError(t, err) dataRows := ignoreBackend.MakeEmptyRows() dataChecksum := verification.MakeKVChecksum(0, 0, 0) @@ -195,26 +198,28 @@ func (s *mysqlSuite) TestWriteRowsErrorOnDup(c *C) { indexChecksum := verification.MakeKVChecksum(0, 0, 0) encoder, err := ignoreBackend.NewEncoder(s.tbl, &kv.SessionOptions{}) - c.Assert(err, IsNil) + require.NoError(t, err) row, err := encoder.Encode(logger, []types.Datum{ types.NewIntDatum(1), - }, 1, []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, -1}, "3.csv", 0) - c.Assert(err, IsNil) + }, 1, []int{0, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1}, "3.csv", 0) + require.NoError(t, err) row.ClassifyAndAppend(&dataRows, &dataChecksum, &indexRows, &indexChecksum) writer, err := engine.LocalWriter(ctx, nil) - c.Assert(err, IsNil) + require.NoError(t, err) err = writer.WriteRows(ctx, []string{"a"}, dataRows) - c.Assert(err, IsNil) + require.NoError(t, err) st, err := writer.Close(ctx) - c.Assert(err, IsNil) - c.Assert(st, IsNil) + require.NoError(t, err) + require.Nil(t, st) } // TODO: temporarily disable this test before we fix strict mode //nolint:unused -func (s *mysqlSuite) testStrictMode(c *C) { +func testStrictMode(t *testing.T) { + s := createMysqlSuite(t) + defer s.TearDownTest(t) ft := *types.NewFieldType(mysql.TypeVarchar) ft.Charset = charset.CharsetUTF8MB4 col0 := &model.ColumnInfo{ID: 1, Name: model.NewCIStr("s0"), State: model.StatePublic, Offset: 0, FieldType: ft} @@ -223,47 +228,52 @@ func (s *mysqlSuite) testStrictMode(c *C) { col1 := &model.ColumnInfo{ID: 2, Name: model.NewCIStr("s1"), State: model.StatePublic, Offset: 1, FieldType: ft} tblInfo := &model.TableInfo{ID: 1, Columns: []*model.ColumnInfo{col0, col1}, PKIsHandle: false, State: model.StatePublic} tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(0), tblInfo) - c.Assert(err, IsNil) + require.NoError(t, err) bk := tidb.NewTiDBBackend(s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig())) encoder, err := bk.NewEncoder(tbl, &kv.SessionOptions{SQLMode: mysql.ModeStrictAllTables}) - c.Assert(err, IsNil) + require.NoError(t, err) logger := log.L() _, err = encoder.Encode(logger, []types.Datum{ types.NewStringDatum("test"), }, 1, []int{0, -1, -1}, "4.csv", 0) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = encoder.Encode(logger, []types.Datum{ types.NewStringDatum("\xff\xff\xff\xff"), }, 1, []int{0, -1, -1}, "5.csv", 0) - c.Assert(err, ErrorMatches, `.*incorrect utf8 value .* for column s0`) + require.Error(t, err) + require.Regexp(t, `.*incorrect utf8 value .* for column s0`, err.Error()) // oepn a new encode because column count changed. encoder, err = bk.NewEncoder(tbl, &kv.SessionOptions{SQLMode: mysql.ModeStrictAllTables}) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = encoder.Encode(logger, []types.Datum{ types.NewStringDatum(""), types.NewStringDatum("非 ASCII 字符串"), }, 1, []int{0, 1, -1}, "6.csv", 0) - c.Assert(err, ErrorMatches, ".*incorrect ascii value .* for column s1") + require.Error(t, err) + require.Regexp(t, ".*incorrect ascii value .* for column s1", err.Error()) } -func (s *mysqlSuite) TestFetchRemoteTableModels_3_x(c *C) { +func TestFetchRemoteTableModels_3_x(t *testing.T) { + t.Parallel() + s := createMysqlSuite(t) + defer s.TearDownTest(t) s.mockDB.ExpectBegin() s.mockDB.ExpectQuery("SELECT version()"). WillReturnRows(sqlmock.NewRows([]string{"version()"}).AddRow("5.7.25-TiDB-v3.0.18")) - s.mockDB.ExpectQuery("\\QSELECT table_name, column_name, column_type, extra FROM information_schema.columns WHERE table_schema = ? ORDER BY table_name, ordinal_position;\\E"). + s.mockDB.ExpectQuery("\\QSELECT table_name, column_name, column_type, generation_expression, extra FROM information_schema.columns WHERE table_schema = ? ORDER BY table_name, ordinal_position;\\E"). WithArgs("test"). - WillReturnRows(sqlmock.NewRows([]string{"table_name", "column_name", "column_type", "extra"}). - AddRow("t", "id", "int(10)", "auto_increment")) + WillReturnRows(sqlmock.NewRows([]string{"table_name", "column_name", "column_type", "generation_expression", "extra"}). + AddRow("t", "id", "int(10)", "", "auto_increment")) s.mockDB.ExpectCommit() bk := tidb.NewTiDBBackend(s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig())) tableInfos, err := bk.FetchRemoteTableModels(context.Background(), "test") - c.Assert(err, IsNil) - c.Assert(tableInfos, DeepEquals, []*model.TableInfo{ + require.NoError(t, err) + require.Equal(t, []*model.TableInfo{ { Name: model.NewCIStr("t"), State: model.StatePublic, @@ -279,17 +289,20 @@ func (s *mysqlSuite) TestFetchRemoteTableModels_3_x(c *C) { }, }, }, - }) + }, tableInfos) } -func (s *mysqlSuite) TestFetchRemoteTableModels_4_0(c *C) { +func TestFetchRemoteTableModels_4_0(t *testing.T) { + t.Parallel() + s := createMysqlSuite(t) + defer s.TearDownTest(t) s.mockDB.ExpectBegin() s.mockDB.ExpectQuery("SELECT version()"). WillReturnRows(sqlmock.NewRows([]string{"version()"}).AddRow("5.7.25-TiDB-v4.0.0")) - s.mockDB.ExpectQuery("\\QSELECT table_name, column_name, column_type, extra FROM information_schema.columns WHERE table_schema = ? ORDER BY table_name, ordinal_position;\\E"). + s.mockDB.ExpectQuery("\\QSELECT table_name, column_name, column_type, generation_expression, extra FROM information_schema.columns WHERE table_schema = ? ORDER BY table_name, ordinal_position;\\E"). WithArgs("test"). - WillReturnRows(sqlmock.NewRows([]string{"table_name", "column_name", "column_type", "extra"}). - AddRow("t", "id", "bigint(20) unsigned", "auto_increment")) + WillReturnRows(sqlmock.NewRows([]string{"table_name", "column_name", "column_type", "generation_expression", "extra"}). + AddRow("t", "id", "bigint(20) unsigned", "", "auto_increment")) s.mockDB.ExpectQuery("SHOW TABLE `test`.`t` NEXT_ROW_ID"). WillReturnRows(sqlmock.NewRows([]string{"DB_NAME", "TABLE_NAME", "COLUMN_NAME", "NEXT_GLOBAL_ROW_ID"}). AddRow("test", "t", "id", int64(1))) @@ -297,8 +310,8 @@ func (s *mysqlSuite) TestFetchRemoteTableModels_4_0(c *C) { bk := tidb.NewTiDBBackend(s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig())) tableInfos, err := bk.FetchRemoteTableModels(context.Background(), "test") - c.Assert(err, IsNil) - c.Assert(tableInfos, DeepEquals, []*model.TableInfo{ + require.NoError(t, err) + require.Equal(t, []*model.TableInfo{ { Name: model.NewCIStr("t"), State: model.StatePublic, @@ -314,17 +327,20 @@ func (s *mysqlSuite) TestFetchRemoteTableModels_4_0(c *C) { }, }, }, - }) + }, tableInfos) } -func (s *mysqlSuite) TestFetchRemoteTableModels_4_x_auto_increment(c *C) { +func TestFetchRemoteTableModels_4_x_auto_increment(t *testing.T) { + t.Parallel() + s := createMysqlSuite(t) + defer s.TearDownTest(t) s.mockDB.ExpectBegin() s.mockDB.ExpectQuery("SELECT version()"). WillReturnRows(sqlmock.NewRows([]string{"version()"}).AddRow("5.7.25-TiDB-v4.0.7")) - s.mockDB.ExpectQuery("\\QSELECT table_name, column_name, column_type, extra FROM information_schema.columns WHERE table_schema = ? ORDER BY table_name, ordinal_position;\\E"). + s.mockDB.ExpectQuery("\\QSELECT table_name, column_name, column_type, generation_expression, extra FROM information_schema.columns WHERE table_schema = ? ORDER BY table_name, ordinal_position;\\E"). WithArgs("test"). - WillReturnRows(sqlmock.NewRows([]string{"table_name", "column_name", "column_type", "extra"}). - AddRow("t", "id", "bigint(20)", "")) + WillReturnRows(sqlmock.NewRows([]string{"table_name", "column_name", "column_type", "generation_expression", "extra"}). + AddRow("t", "id", "bigint(20)", "", "")) s.mockDB.ExpectQuery("SHOW TABLE `test`.`t` NEXT_ROW_ID"). WillReturnRows(sqlmock.NewRows([]string{"DB_NAME", "TABLE_NAME", "COLUMN_NAME", "NEXT_GLOBAL_ROW_ID", "ID_TYPE"}). AddRow("test", "t", "id", int64(1), "AUTO_INCREMENT")) @@ -332,8 +348,8 @@ func (s *mysqlSuite) TestFetchRemoteTableModels_4_x_auto_increment(c *C) { bk := tidb.NewTiDBBackend(s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig())) tableInfos, err := bk.FetchRemoteTableModels(context.Background(), "test") - c.Assert(err, IsNil) - c.Assert(tableInfos, DeepEquals, []*model.TableInfo{ + require.NoError(t, err) + require.Equal(t, []*model.TableInfo{ { Name: model.NewCIStr("t"), State: model.StatePublic, @@ -349,17 +365,20 @@ func (s *mysqlSuite) TestFetchRemoteTableModels_4_x_auto_increment(c *C) { }, }, }, - }) + }, tableInfos) } -func (s *mysqlSuite) TestFetchRemoteTableModels_4_x_auto_random(c *C) { +func TestFetchRemoteTableModels_4_x_auto_random(t *testing.T) { + t.Parallel() + s := createMysqlSuite(t) + defer s.TearDownTest(t) s.mockDB.ExpectBegin() s.mockDB.ExpectQuery("SELECT version()"). WillReturnRows(sqlmock.NewRows([]string{"version()"}).AddRow("5.7.25-TiDB-v4.0.7")) - s.mockDB.ExpectQuery("\\QSELECT table_name, column_name, column_type, extra FROM information_schema.columns WHERE table_schema = ? ORDER BY table_name, ordinal_position;\\E"). + s.mockDB.ExpectQuery("\\QSELECT table_name, column_name, column_type, generation_expression, extra FROM information_schema.columns WHERE table_schema = ? ORDER BY table_name, ordinal_position;\\E"). WithArgs("test"). - WillReturnRows(sqlmock.NewRows([]string{"table_name", "column_name", "column_type", "extra"}). - AddRow("t", "id", "bigint(20)", "")) + WillReturnRows(sqlmock.NewRows([]string{"table_name", "column_name", "column_type", "generation_expression", "extra"}). + AddRow("t", "id", "bigint(20)", "1 + 2", "")) s.mockDB.ExpectQuery("SHOW TABLE `test`.`t` NEXT_ROW_ID"). WillReturnRows(sqlmock.NewRows([]string{"DB_NAME", "TABLE_NAME", "COLUMN_NAME", "NEXT_GLOBAL_ROW_ID", "ID_TYPE"}). AddRow("test", "t", "id", int64(1), "AUTO_RANDOM")) @@ -367,8 +386,8 @@ func (s *mysqlSuite) TestFetchRemoteTableModels_4_x_auto_random(c *C) { bk := tidb.NewTiDBBackend(s.dbHandle, config.ErrorOnDup, errormanager.New(nil, config.NewConfig())) tableInfos, err := bk.FetchRemoteTableModels(context.Background(), "test") - c.Assert(err, IsNil) - c.Assert(tableInfos, DeepEquals, []*model.TableInfo{ + require.NoError(t, err) + require.Equal(t, []*model.TableInfo{ { Name: model.NewCIStr("t"), State: model.StatePublic, @@ -382,15 +401,18 @@ func (s *mysqlSuite) TestFetchRemoteTableModels_4_x_auto_random(c *C) { FieldType: types.FieldType{ Flag: mysql.PriKeyFlag, }, + GeneratedExprString: "1 + 2", }, }, }, - }) + }, tableInfos) } -func (s *mysqlSuite) TestWriteRowsErrorDowngrading(c *C) { +func TestWriteRowsErrorDowngrading(t *testing.T) { + t.Parallel() nonRetryableError := sql.ErrNoRows - + s := createMysqlSuite(t) + defer s.TearDownTest(t) // First, batch insert, fail and rollback. s.mockDB. ExpectExec("\\QINSERT INTO `foo`.`bar`(`a`) VALUES(1),(2),(3),(4),(5)\\E"). @@ -436,7 +458,7 @@ func (s *mysqlSuite) TestWriteRowsErrorDowngrading(c *C) { }), ) engine, err := ignoreBackend.OpenEngine(ctx, &backend.EngineConfig{}, "`foo`.`bar`", 1) - c.Assert(err, IsNil) + require.NoError(t, err) dataRows := ignoreBackend.MakeEmptyRows() dataChecksum := verification.MakeKVChecksum(0, 0, 0) @@ -444,47 +466,47 @@ func (s *mysqlSuite) TestWriteRowsErrorDowngrading(c *C) { indexChecksum := verification.MakeKVChecksum(0, 0, 0) encoder, err := ignoreBackend.NewEncoder(s.tbl, &kv.SessionOptions{}) - c.Assert(err, IsNil) + require.NoError(t, err) row, err := encoder.Encode(logger, []types.Datum{ types.NewIntDatum(1), - }, 1, []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, -1}, "7.csv", 0) - c.Assert(err, IsNil) + }, 1, []int{0, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1}, "7.csv", 0) + require.NoError(t, err) row.ClassifyAndAppend(&dataRows, &dataChecksum, &indexRows, &indexChecksum) row, err = encoder.Encode(logger, []types.Datum{ types.NewIntDatum(2), - }, 1, []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, -1}, "8.csv", 0) - c.Assert(err, IsNil) + }, 1, []int{0, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1}, "8.csv", 0) + require.NoError(t, err) row.ClassifyAndAppend(&dataRows, &dataChecksum, &indexRows, &indexChecksum) row, err = encoder.Encode(logger, []types.Datum{ types.NewIntDatum(3), - }, 1, []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, -1}, "9.csv", 0) - c.Assert(err, IsNil) + }, 1, []int{0, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1}, "9.csv", 0) + require.NoError(t, err) row.ClassifyAndAppend(&dataRows, &dataChecksum, &indexRows, &indexChecksum) row, err = encoder.Encode(logger, []types.Datum{ types.NewIntDatum(4), - }, 1, []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, -1}, "10.csv", 0) - c.Assert(err, IsNil) + }, 1, []int{0, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1}, "10.csv", 0) + require.NoError(t, err) row.ClassifyAndAppend(&dataRows, &dataChecksum, &indexRows, &indexChecksum) row, err = encoder.Encode(logger, []types.Datum{ types.NewIntDatum(5), - }, 1, []int{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, -1}, "11.csv", 0) - c.Assert(err, IsNil) + }, 1, []int{0, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1}, "11.csv", 0) + require.NoError(t, err) row.ClassifyAndAppend(&dataRows, &dataChecksum, &indexRows, &indexChecksum) writer, err := engine.LocalWriter(ctx, nil) - c.Assert(err, IsNil) + require.NoError(t, err) err = writer.WriteRows(ctx, []string{"a"}, dataRows) - c.Assert(err, NotNil) + require.Error(t, err) st, err := writer.Close(ctx) - c.Assert(err, IsNil) - c.Assert(st, IsNil) + require.NoError(t, err) + require.Nil(t, st) } diff --git a/br/pkg/lightning/config/config.go b/br/pkg/lightning/config/config.go index 8f2e6f2dfa9ac..027404a48bb1a 100644 --- a/br/pkg/lightning/config/config.go +++ b/br/pkg/lightning/config/config.go @@ -473,6 +473,14 @@ type IgnoreColumns struct { Columns []string `toml:"columns" json:"columns"` } +func (ic *IgnoreColumns) ColumnsMap() map[string]struct{} { + columnMap := make(map[string]struct{}, len(ic.Columns)) + for _, c := range ic.Columns { + columnMap[c] = struct{}{} + } + return columnMap +} + // GetIgnoreColumns gets Ignore config by schema name/regex and table name/regex. func (igCols AllIgnoreColumns) GetIgnoreColumns(db string, table string, caseSensitive bool) (*IgnoreColumns, error) { if !caseSensitive { diff --git a/br/pkg/lightning/config/config_test.go b/br/pkg/lightning/config/config_test.go index 1e7e751b20b3d..581d4e67f8bfa 100644 --- a/br/pkg/lightning/config/config_test.go +++ b/br/pkg/lightning/config/config_test.go @@ -581,6 +581,13 @@ func (s *configTestSuite) TestLoadConfig(c *C) { result := taskCfg.String() c.Assert(result, Matches, `.*"pd-addr":"172.16.30.11:2379,172.16.30.12:2379".*`) + + cfg, err = config.LoadGlobalConfig([]string{}, nil) + c.Assert(err, IsNil) + c.Assert(cfg.App.Config.File, Matches, ".*lightning.log.*") + cfg, err = config.LoadGlobalConfig([]string{"--log-file", "-"}, nil) + c.Assert(err, IsNil) + c.Assert(cfg.App.Config.File, Equals, "-") } func (s *configTestSuite) TestDefaultImporterBackendValue(c *C) { diff --git a/br/pkg/lightning/config/global.go b/br/pkg/lightning/config/global.go index 7eb8e240c9dfe..d9bd80ef4139a 100644 --- a/br/pkg/lightning/config/global.go +++ b/br/pkg/lightning/config/global.go @@ -200,10 +200,7 @@ func LoadGlobalConfig(args []string, extraFlags func(*flag.FlagSet)) (*GlobalCon if *logFilePath != "" { cfg.App.Config.File = *logFilePath } - // "-" is a special config for log to stdout - if cfg.App.Config.File == "-" { - cfg.App.Config.File = "" - } else if cfg.App.Config.File == "" { + if cfg.App.Config.File == "" { cfg.App.Config.File = timestampLogFileName() } if *tidbHost != "" { diff --git a/br/pkg/lightning/lightning_server_serial_test.go b/br/pkg/lightning/lightning_server_serial_test.go index efede7cb9d581..e8a9d86ebbdec 100644 --- a/br/pkg/lightning/lightning_server_serial_test.go +++ b/br/pkg/lightning/lightning_server_serial_test.go @@ -306,6 +306,7 @@ func TestHTTPAPIOutsideServerMode(t *testing.T) { errCh := make(chan error) cfg := config.NewConfig() + cfg.TiDB.DistSQLScanConcurrency = 4 err := cfg.LoadFromGlobal(s.lightning.globalCfg) require.NoError(t, err) go func() { diff --git a/br/pkg/lightning/log/log.go b/br/pkg/lightning/log/log.go index 2dc24acac1541..8521cf85a6579 100644 --- a/br/pkg/lightning/log/log.go +++ b/br/pkg/lightning/log/log.go @@ -93,8 +93,8 @@ func InitLogger(cfg *Config, tidbLoglevel string) error { // Filter logs from TiDB and PD. return NewFilterCore(core, "github.com/tikv/pd/") }) - - if len(cfg.File) > 0 { + // "-" is a special config for log to stdout. + if len(cfg.File) > 0 && cfg.File != "-" { logCfg.File = pclog.FileLogConfig{ Filename: cfg.File, MaxSize: cfg.FileMaxSize, diff --git a/br/pkg/lightning/log/log_serial_test.go b/br/pkg/lightning/log/log_serial_test.go new file mode 100644 index 0000000000000..63ef2bf321ab1 --- /dev/null +++ b/br/pkg/lightning/log/log_serial_test.go @@ -0,0 +1,43 @@ +package log_test + +import ( + "io" + "os" + "testing" + + "github.com/pingcap/tidb/br/pkg/lightning/log" + "github.com/stretchr/testify/require" +) + +func TestInitStdoutLogger(t *testing.T) { + r, w, err := os.Pipe() + require.NoError(t, err) + oldStdout := os.Stdout + os.Stdout = w + + msg := "logger is initialized to stdout" + outputC := make(chan string, 1) + go func() { + buf := make([]byte, 4096) + n := 0 + for { + nn, err := r.Read(buf[n:]) + if nn == 0 || err == io.EOF { + break + } + require.NoError(t, err) + n += nn + } + outputC <- string(buf[:n]) + }() + + logCfg := &log.Config{File: "-"} + log.InitLogger(logCfg, "info") + log.L().Info(msg) + + os.Stdout = oldStdout + require.NoError(t, w.Close()) + output := <-outputC + require.NoError(t, r.Close()) + require.Contains(t, output, msg) +} diff --git a/br/pkg/lightning/restore/check_info.go b/br/pkg/lightning/restore/check_info.go index 1b86ee482f362..a4e49a2fbc274 100644 --- a/br/pkg/lightning/restore/check_info.go +++ b/br/pkg/lightning/restore/check_info.go @@ -41,7 +41,9 @@ import ( "github.com/pingcap/tidb/br/pkg/version" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" + "github.com/pingcap/tidb/types" "github.com/tikv/pd/server/api" pdconfig "github.com/tikv/pd/server/config" @@ -577,7 +579,7 @@ func hasDefault(col *model.ColumnInfo) bool { col.IsGenerated() || mysql.HasAutoIncrementFlag(col.Flag) } -func (rc *Controller) readColumnsAndCount(ctx context.Context, dataFileMeta mydump.SourceFileMeta) (cols []string, colCnt int, err error) { +func (rc *Controller) readFirstRow(ctx context.Context, dataFileMeta mydump.SourceFileMeta) (cols []string, row []types.Datum, err error) { var reader storage.ReadSeekCloser if dataFileMeta.Type == mydump.SourceTypeParquet { reader, err = mydump.OpenParquetReader(ctx, rc.store, dataFileMeta.Path, dataFileMeta.FileSize) @@ -585,7 +587,7 @@ func (rc *Controller) readColumnsAndCount(ctx context.Context, dataFileMeta mydu reader, err = rc.store.Open(ctx, dataFileMeta.Path) } if err != nil { - return nil, 0, errors.Trace(err) + return nil, nil, errors.Trace(err) } var parser mydump.Parser @@ -596,18 +598,18 @@ func (rc *Controller) readColumnsAndCount(ctx context.Context, dataFileMeta mydu // Create a utf8mb4 convertor to encode and decode data with the charset of CSV files. charsetConvertor, err := mydump.NewCharsetConvertor(rc.cfg.Mydumper.DataCharacterSet, rc.cfg.Mydumper.DataInvalidCharReplace) if err != nil { - return nil, 0, errors.Trace(err) + return nil, nil, errors.Trace(err) } parser, err = mydump.NewCSVParser(&rc.cfg.Mydumper.CSV, reader, blockBufSize, rc.ioWorkers, hasHeader, charsetConvertor) if err != nil { - return nil, 0, errors.Trace(err) + return nil, nil, errors.Trace(err) } case mydump.SourceTypeSQL: parser = mydump.NewChunkParser(rc.cfg.TiDB.SQLMode, reader, blockBufSize, rc.ioWorkers) case mydump.SourceTypeParquet: parser, err = mydump.NewParquetParser(ctx, rc.store, reader, dataFileMeta.Path) if err != nil { - return nil, 0, errors.Trace(err) + return nil, nil, errors.Trace(err) } default: panic(fmt.Sprintf("unknown file type '%s'", dataFileMeta.Type)) @@ -616,13 +618,18 @@ func (rc *Controller) readColumnsAndCount(ctx context.Context, dataFileMeta mydu err = parser.ReadRow() if err != nil && errors.Cause(err) != io.EOF { - return nil, 0, errors.Trace(err) + return nil, nil, errors.Trace(err) } - return parser.Columns(), len(parser.LastRow().Row), nil + return parser.Columns(), parser.LastRow().Row, nil } // SchemaIsValid checks the import file and cluster schema is match. func (rc *Controller) SchemaIsValid(ctx context.Context, tableInfo *mydump.MDTableMeta) ([]string, error) { + if len(tableInfo.DataFiles) == 0 { + log.L().Info("no data files detected", zap.String("db", tableInfo.DB), zap.String("table", tableInfo.Name)) + return nil, nil + } + msgs := make([]string, 0) info, ok := rc.dbInfos[tableInfo.DB].Tables[tableInfo.Name] if !ok { @@ -631,19 +638,11 @@ func (rc *Controller) SchemaIsValid(ctx context.Context, tableInfo *mydump.MDTab return msgs, nil } - igCols := make(map[string]struct{}) igCol, err := rc.cfg.Mydumper.IgnoreColumns.GetIgnoreColumns(tableInfo.DB, tableInfo.Name, rc.cfg.Mydumper.CaseSensitive) if err != nil { return nil, errors.Trace(err) } - for _, col := range igCol.Columns { - igCols[col] = struct{}{} - } - - if len(tableInfo.DataFiles) == 0 { - log.L().Info("no data files detected", zap.String("db", tableInfo.DB), zap.String("table", tableInfo.Name)) - return nil, nil - } + igCols := igCol.ColumnsMap() colCountFromTiDB := len(info.Core.Columns) core := info.Core @@ -658,86 +657,264 @@ func (rc *Controller) SchemaIsValid(ctx context.Context, tableInfo *mydump.MDTab defaultCols[model.ExtraHandleName.String()] = struct{}{} // only check the first file of this table. - if len(tableInfo.DataFiles) > 0 { - dataFile := tableInfo.DataFiles[0] - log.L().Info("datafile to check", zap.String("db", tableInfo.DB), - zap.String("table", tableInfo.Name), zap.String("path", dataFile.FileMeta.Path)) - // get columns name from data file. - dataFileMeta := dataFile.FileMeta - - if tp := dataFileMeta.Type; tp != mydump.SourceTypeCSV && tp != mydump.SourceTypeSQL && tp != mydump.SourceTypeParquet { - msgs = append(msgs, fmt.Sprintf("file '%s' with unknown source type '%s'", dataFileMeta.Path, dataFileMeta.Type.String())) - return msgs, nil - } - colsFromDataFile, colCountFromDataFile, err := rc.readColumnsAndCount(ctx, dataFileMeta) - if err != nil { - return nil, errors.Trace(err) - } - if colsFromDataFile == nil && colCountFromDataFile == 0 { - log.L().Info("file contains no data, skip checking against schema validity", zap.String("path", dataFileMeta.Path)) - return msgs, nil + dataFile := tableInfo.DataFiles[0] + log.L().Info("datafile to check", zap.String("db", tableInfo.DB), + zap.String("table", tableInfo.Name), zap.String("path", dataFile.FileMeta.Path)) + // get columns name from data file. + dataFileMeta := dataFile.FileMeta + + if tp := dataFileMeta.Type; tp != mydump.SourceTypeCSV && tp != mydump.SourceTypeSQL && tp != mydump.SourceTypeParquet { + msgs = append(msgs, fmt.Sprintf("file '%s' with unknown source type '%s'", dataFileMeta.Path, dataFileMeta.Type.String())) + return msgs, nil + } + colsFromDataFile, row, err := rc.readFirstRow(ctx, dataFileMeta) + if err != nil { + return nil, errors.Trace(err) + } + if colsFromDataFile == nil && len(row) == 0 { + log.L().Info("file contains no data, skip checking against schema validity", zap.String("path", dataFileMeta.Path)) + return msgs, nil + } + + if colsFromDataFile == nil { + // when there is no columns name in data file. we must insert data in order. + // so the last several columns either can be ignored or has a default value. + for i := len(row); i < colCountFromTiDB; i++ { + if _, ok := defaultCols[core.Columns[i].Name.L]; !ok { + msgs = append(msgs, fmt.Sprintf("TiDB schema `%s`.`%s` has %d columns,"+ + "and data file has %d columns, but column %s are missing the default value,"+ + "please give column a default value to skip this check", + tableInfo.DB, tableInfo.Name, colCountFromTiDB, len(row), core.Columns[i].Name.L)) + } } + return msgs, nil + } - if colsFromDataFile == nil { - // when there is no columns name in data file. we must insert data in order. - // so the last several columns either can be ignored or has a default value. - for i := colCountFromDataFile; i < colCountFromTiDB; i++ { - if _, ok := defaultCols[core.Columns[i].Name.L]; !ok { - msgs = append(msgs, fmt.Sprintf("TiDB schema `%s`.`%s` has %d columns,"+ - "and data file has %d columns, but column %s are missing the default value,"+ - "please give column a default value to skip this check", - tableInfo.DB, tableInfo.Name, colCountFromTiDB, colCountFromDataFile, core.Columns[i].Name.L)) - } + // compare column names and make sure + // 1. TiDB table info has data file's all columns(besides ignore columns) + // 2. Those columns not introduced in data file always have a default value. + colMap := make(map[string]struct{}) + for col := range igCols { + colMap[col] = struct{}{} + } + for _, col := range core.Columns { + if _, ok := colMap[col.Name.L]; ok { + // tidb's column is ignored + // we need ensure this column has the default value. + if _, hasDefault := defaultCols[col.Name.L]; !hasDefault { + msgs = append(msgs, fmt.Sprintf("TiDB schema `%s`.`%s`'s column %s cannot be ignored,"+ + "because it doesn't have a default value, please set tables.ignoreColumns properly", + tableInfo.DB, tableInfo.Name, col.Name.L)) } } else { - // compare column names and make sure - // 1. TiDB table info has data file's all columns(besides ignore columns) - // 2. Those columns not introduced in data file always have a default value. - colMap := make(map[string]struct{}) - for col := range igCols { - colMap[col] = struct{}{} + colMap[col.Name.L] = struct{}{} + } + } + // tidb_rowid can be ignored in check + colMap[model.ExtraHandleName.String()] = struct{}{} + for _, col := range colsFromDataFile { + if _, ok := colMap[col]; !ok { + checkMsg := "please check table schema" + if dataFileMeta.Type == mydump.SourceTypeCSV && rc.cfg.Mydumper.CSV.Header { + checkMsg += " and csv file header" } - for _, col := range core.Columns { - if _, ok := colMap[col.Name.L]; ok { - // tidb's column is ignored - // we need ensure this column has the default value. - if _, hasDefault := defaultCols[col.Name.L]; !hasDefault { - msgs = append(msgs, fmt.Sprintf("TiDB schema `%s`.`%s`'s column %s cannot be ignored,"+ - "because it doesn't hava a default value, please set tables.ignoreColumns properly", - tableInfo.DB, tableInfo.Name, col.Name.L)) - } - } else { - colMap[col.Name.L] = struct{}{} - } + msgs = append(msgs, fmt.Sprintf("TiDB schema `%s`.`%s` doesn't have column %s, "+ + "%s or use tables.ignoreColumns to ignore %s", + tableInfo.DB, tableInfo.Name, col, checkMsg, col)) + } else { + // remove column for next iteration + delete(colMap, col) + } + } + // if theses rest columns don't have a default value. + for col := range colMap { + if _, ok := defaultCols[col]; ok { + continue + } + msgs = append(msgs, fmt.Sprintf("TiDB schema `%s`.`%s` doesn't have the default value for %s"+ + "please give a default value for %s or choose another column to ignore or add this column in data file", + tableInfo.DB, tableInfo.Name, col, col)) + } + return msgs, nil +} + +// checkCSVHeader try to check whether the csv header config is consistent with the source csv files by: +// 1. pick one table with two CSV files and a unique/primary key +// 2. read the first row of those two CSV files +// 3. checks if the content of those first rows are compatible with the table schema, and whether the +// two rows are identical, to determine if the first rows are a header rows. +func (rc *Controller) checkCSVHeader(ctx context.Context, dbMetas []*mydump.MDDatabaseMeta) error { + // if cfg set header = ture but source files actually contain not header, former SchemaCheck should + // return error in this situation, so we need do it again. + if rc.cfg.Mydumper.CSV.Header { + return nil + } + var ( + tableMeta *mydump.MDTableMeta + csvCount int + hasUniqueIdx bool + ) + // only check one table source files for better performance. The checked table is chosen based on following two factor: + // 1. contains at least 1 csv source file, 2 is preferable + // 2. table schema contains primary key or unique key + // if the two factors can't be both satisfied, the first one has a higher priority +outer: + for _, dbMeta := range dbMetas { + for _, tblMeta := range dbMeta.Tables { + if len(tblMeta.DataFiles) == 0 { + continue } - // tidb_rowid can be ignored in check - colMap[model.ExtraHandleName.String()] = struct{}{} - for _, col := range colsFromDataFile { - if _, ok := colMap[col]; !ok { - checkMsg := "please check table schema" - if dataFileMeta.Type == mydump.SourceTypeCSV && rc.cfg.Mydumper.CSV.Header { - checkMsg += " and csv file header" + tableHasUniqueIdx := false + tableCSVCount := 0 + for _, f := range tblMeta.DataFiles { + if f.FileMeta.Type == mydump.SourceTypeCSV { + tableCSVCount++ + if tableCSVCount >= 2 { + break } - msgs = append(msgs, fmt.Sprintf("TiDB schema `%s`.`%s` doesn't have column %s, "+ - "%s or use tables.ignoreColumns to ignore %s", - tableInfo.DB, tableInfo.Name, col, checkMsg, col)) - } else { - // remove column for next iteration - delete(colMap, col) } } - // if theses rest columns don't have a default value. - for col := range colMap { - if _, ok := defaultCols[col]; ok { - continue + if tableCSVCount == 0 { + continue + } + + info := rc.dbInfos[tblMeta.DB].Tables[tblMeta.Name] + for _, idx := range info.Core.Indices { + if idx.Primary || idx.Unique { + tableHasUniqueIdx = true } - msgs = append(msgs, fmt.Sprintf("TiDB schema `%s`.`%s` doesn't have the default value for %s"+ - "please give a default value for %s or choose another column to ignore or add this column in data file", - tableInfo.DB, tableInfo.Name, col, col)) + } + + if tableCSVCount >= 2 && hasUniqueIdx { + tableMeta = tblMeta + csvCount = tableCSVCount + hasUniqueIdx = tableHasUniqueIdx + // if a perfect table source is found, we can stop check more tables + break outer + } + if tableCSVCount > csvCount || (tableCSVCount == csvCount && !hasUniqueIdx && tableHasUniqueIdx) { + tableMeta = tblMeta + csvCount = tableCSVCount + hasUniqueIdx = tableHasUniqueIdx } } } - return msgs, nil + + if tableMeta == nil { + return nil + } + + var rows [][]types.Datum + for _, f := range tableMeta.DataFiles { + if f.FileMeta.Type != mydump.SourceTypeCSV { + continue + } + _, row, err := rc.readFirstRow(ctx, f.FileMeta) + if err != nil { + return errors.Trace(err) + } + if len(row) > 0 { + rows = append(rows, row) + } + // only check at most two of all the files + if len(rows) >= 2 { + break + } + } + if len(rows) == 0 { + return nil + } else if len(rows) >= 2 { + // if the first row in two source files are not the same, they should not be the header line + // NOTE: though lightning's logic allows different source files contains different columns or the + // order is difference, here we only check if they are exactly the same because this is the common case. + if len(rows[0]) != len(rows[1]) { + return nil + } + + for i := 0; i < len(rows[0]); i++ { + if rows[0][i].GetString() != rows[1][i].GetString() { + return nil + } + } + } + + // check if some fields are unique and not ignored + // if at least one field appears in a unique key, we can sure there is something wrong, + // they should be either the header line or the data is duplicated. + tableInfo := rc.dbInfos[tableMeta.DB].Tables[tableMeta.Name] + tableFields := make(map[string]struct{}) + uniqueIdxFields := make(map[string]struct{}) + ignoreColumns, err := rc.cfg.Mydumper.IgnoreColumns.GetIgnoreColumns(tableMeta.DB, tableMeta.Name, rc.cfg.Mydumper.CaseSensitive) + if err != nil { + return errors.Trace(err) + } + ignoreColsSet := make(map[string]struct{}) + for _, col := range ignoreColumns.Columns { + ignoreColsSet[col] = struct{}{} + } + for _, idx := range tableInfo.Core.Indices { + if !idx.Unique && !idx.Primary { + continue + } + for _, col := range idx.Columns { + if _, ok := ignoreColsSet[col.Name.L]; !ok { + uniqueIdxFields[col.Name.L] = struct{}{} + } + } + } + for _, f := range tableInfo.Core.Columns { + tableFields[f.Name.L] = struct{}{} + } + if common.TableHasAutoRowID(tableInfo.Core) { + tableFields[model.ExtraHandleName.L] = struct{}{} + } + hasUniqueField := false + for _, d := range rows[0] { + val := strings.ToLower(d.GetString()) + if _, ok := tableFields[val]; !ok { + return nil + } + if _, ok := uniqueIdxFields[val]; ok { + hasUniqueField = true + break + } + } + + msg := fmt.Sprintf("source csv files contains header row but `mydumper.csv.header` is false, checked table is `%s`.`%s`", + tableMeta.DB, tableMeta.Name) + level := Warn + if hasUniqueField && len(rows) > 1 { + level = Critical + } else if !checkFieldCompatibility(tableInfo.Core, ignoreColsSet, rows[0]) { + // if there are only 1 csv file or there is not unique key, try to check if all columns are compatible with string value + level = Critical + } + rc.checkTemplate.Collect(level, false, msg) + + return nil +} + +func checkFieldCompatibility(tbl *model.TableInfo, ignoreCols map[string]struct{}, values []types.Datum) bool { + se := kv.NewSession(&kv.SessionOptions{ + SQLMode: mysql.ModeStrictTransTables, + }) + for i, col := range tbl.Columns { + // do not check ignored columns + if _, ok := ignoreCols[col.Name.L]; ok { + continue + } + if i >= len(values) { + break + } + _, err := table.CastValue(se, values[i], col, true, false) + if err != nil { + log.L().Error("field value is not consistent with column type", zap.String("value", values[i].GetString()), + zap.Any("column_info", col), zap.Error(err)) + return false + } + } + + return true } func (rc *Controller) sampleDataFromTable(ctx context.Context, dbName string, tableMeta *mydump.MDTableMeta, tableInfo *model.TableInfo) error { @@ -816,7 +993,7 @@ outloop: case nil: if !initializedColumns { if len(columnPermutation) == 0 { - columnPermutation, err = createColumnPermutation(columnNames, igCols.Columns, tableInfo) + columnPermutation, err = createColumnPermutation(columnNames, igCols.ColumnsMap(), tableInfo) if err != nil { return errors.Trace(err) } diff --git a/br/pkg/lightning/restore/check_info_test.go b/br/pkg/lightning/restore/check_info_test.go new file mode 100644 index 0000000000000..e1dd939d9c2b1 --- /dev/null +++ b/br/pkg/lightning/restore/check_info_test.go @@ -0,0 +1,405 @@ +// 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 restore + +import ( + "context" + "fmt" + "os" + "path/filepath" + + . "github.com/pingcap/check" + + "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" + "github.com/pingcap/tidb/br/pkg/lightning/config" + "github.com/pingcap/tidb/br/pkg/lightning/mydump" + "github.com/pingcap/tidb/br/pkg/lightning/worker" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + tmock "github.com/pingcap/tidb/util/mock" +) + +var _ = Suite(&checkInfoSuite{}) + +type checkInfoSuite struct{} + +const passed CheckType = "pass" + +func (s *checkInfoSuite) TestCheckCSVHeader(c *C) { + dir := c.MkDir() + ctx := context.Background() + mockStore, err := storage.NewLocalStorage(dir) + c.Assert(err, IsNil) + + type tableSource struct { + Name string + SQL string + Sources []string + } + + cases := []struct { + ignoreColumns []*config.IgnoreColumns + // empty msg means check pass + level CheckType + Sources map[string][]*tableSource + }{ + + { + nil, + + passed, + map[string][]*tableSource{ + "db": { + { + "tbl1", + "create table tbl1 (a varchar(16), b varchar(8))", + []string{ + "aa,b\r\n", + }, + }, + }, + }, + }, + { + nil, + + passed, + map[string][]*tableSource{ + "db": { + { + "tbl1", + "create table tbl1 (a varchar(16), b varchar(8))", + []string{ + "a,b\r\ntest1,test2\r\n", + "aa,b\r\n", + }, + }, + }, + }, + }, + { + nil, + + Warn, + map[string][]*tableSource{ + "db": { + { + "tbl1", + "create table tbl1 (a varchar(16), b varchar(8))", + []string{ + "a,b\r\n", + }, + }, + }, + }, + }, + { + nil, + + Warn, + map[string][]*tableSource{ + "db": { + { + "tbl1", + "create table tbl1 (a varchar(16), b varchar(8))", + []string{ + "a,b\r\ntest1,test2\r\n", + "a,b\r\ntest3,test4\n", + }, + }, + }, + }, + }, + { + nil, + + Warn, + map[string][]*tableSource{ + "db": { + { + "tbl1", + "create table tbl1 (a varchar(16), b varchar(8), PRIMARY KEY (`a`))", + []string{ + "a,b\r\ntest1,test2\r\n", + }, + }, + }, + }, + }, + { + nil, + + Critical, + map[string][]*tableSource{ + "db": { + { + "tbl1", + "create table tbl1 (a varchar(16), b varchar(8), PRIMARY KEY (`a`))", + []string{ + "a,b\r\ntest1,test2\r\n", + "a,b\r\ntest3,test4\r\n", + }, + }, + }, + }, + }, + // ignore primary key, should still be warn + { + []*config.IgnoreColumns{ + { + DB: "db", + Table: "tbl1", + Columns: []string{"a"}, + }, + }, + Warn, + map[string][]*tableSource{ + "db": { + { + "tbl1", + "create table tbl1 (a varchar(16), b varchar(8), PRIMARY KEY (`a`))", + []string{ + "a,b\r\ntest1,test2\r\n", + "a,b\r\ntest3,test4\r\n", + }, + }, + }, + }, + }, + // ignore primary key, but has other unique key + { + []*config.IgnoreColumns{ + { + DB: "db", + Table: "tbl1", + Columns: []string{"a"}, + }, + }, + Critical, + map[string][]*tableSource{ + "db": { + { + "tbl1", + "create table tbl1 (a varchar(16), b varchar(8), PRIMARY KEY (`a`), unique key uk (`b`))", + []string{ + "a,b\r\ntest1,test2\r\n", + "a,b\r\ntest3,test4\r\n", + }, + }, + }, + }, + }, + // ignore primary key, non other unique key + { + []*config.IgnoreColumns{ + { + DB: "db", + Table: "tbl1", + Columns: []string{"a"}, + }, + }, + Warn, + map[string][]*tableSource{ + "db": { + { + "tbl1", + "create table tbl1 (a varchar(16), b varchar(8), PRIMARY KEY (`a`), KEY idx_b (`b`))", + []string{ + "a,b\r\ntest1,test2\r\n", + "a,b\r\ntest3,test4\r\n", + }, + }, + }, + }, + }, + // non unique key, but data type inconsistent + { + nil, + Critical, + map[string][]*tableSource{ + "db": { + { + "tbl1", + "create table tbl1 (a bigint, b varchar(8));", + []string{ + "a,b\r\ntest1,test2\r\n", + "a,b\r\ntest3,test4\r\n", + }, + }, + }, + }, + }, + // non unique key, but ignore inconsistent field + { + []*config.IgnoreColumns{ + { + DB: "db", + Table: "tbl1", + Columns: []string{"a"}, + }, + }, + Warn, + map[string][]*tableSource{ + "db": { + { + "tbl1", + "create table tbl1 (a bigint, b varchar(8));", + []string{ + "a,b\r\ntest1,test2\r\n", + "a,b\r\ntest3,test4\r\n", + }, + }, + }, + }, + }, + // multiple tables, test the choose priority + { + nil, + Critical, + map[string][]*tableSource{ + "db": { + { + "tbl1", + "create table tbl1 (a varchar(8), b varchar(8));", + []string{ + "a,b\r\ntest1,test2\r\n", + }, + }, + { + "tbl2", + "create table tbl1 (a varchar(8) primary key, b varchar(8));", + []string{ + "a,b\r\ntest1,test2\r\n", + "a,b\r\ntest3,test4\r\n", + }, + }, + }, + }, + }, + { + nil, + Critical, + map[string][]*tableSource{ + "db": { + { + "tbl1", + "create table tbl1 (a varchar(8), b varchar(8));", + []string{ + "a,b\r\ntest1,test2\r\n", + }, + }, + }, + "db2": { + { + "tbl2", + "create table tbl1 (a bigint, b varchar(8));", + []string{ + "a,b\r\ntest1,test2\r\n", + "a,b\r\ntest3,test4\r\n", + }, + }, + }, + }, + }, + } + + cfg := &config.Config{ + Mydumper: config.MydumperRuntime{ + ReadBlockSize: config.ReadBlockSize, + CSV: config.CSVConfig{ + Separator: ",", + Delimiter: `"`, + Header: false, + NotNull: false, + Null: `\N`, + BackslashEscape: true, + TrimLastSep: false, + }, + }, + } + rc := &Controller{ + cfg: cfg, + store: mockStore, + ioWorkers: worker.NewPool(context.Background(), 1, "io"), + } + + p := parser.New() + p.SetSQLMode(mysql.ModeANSIQuotes) + se := tmock.NewContext() + + for _, ca := range cases { + rc.checkTemplate = NewSimpleTemplate() + cfg.Mydumper.IgnoreColumns = ca.ignoreColumns + rc.dbInfos = make(map[string]*checkpoints.TidbDBInfo) + + dbMetas := make([]*mydump.MDDatabaseMeta, 0) + for db, tbls := range ca.Sources { + tblMetas := make([]*mydump.MDTableMeta, 0, len(tbls)) + dbInfo := &checkpoints.TidbDBInfo{ + Name: db, + Tables: make(map[string]*checkpoints.TidbTableInfo), + } + rc.dbInfos[db] = dbInfo + + for _, tbl := range tbls { + node, err := p.ParseOneStmt(tbl.SQL, "", "") + c.Assert(err, IsNil) + core, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 0xabcdef) + c.Assert(err, IsNil) + core.State = model.StatePublic + dbInfo.Tables[tbl.Name] = &checkpoints.TidbTableInfo{ + ID: core.ID, + DB: db, + Name: tbl.Name, + Core: core, + } + + fileInfos := make([]mydump.FileInfo, 0, len(tbl.Sources)) + for i, s := range tbl.Sources { + fileName := fmt.Sprintf("%s.%s.%d.csv", db, tbl.Name, i) + err = os.WriteFile(filepath.Join(dir, fileName), []byte(s), 0o644) + c.Assert(err, IsNil) + fileInfos = append(fileInfos, mydump.FileInfo{ + FileMeta: mydump.SourceFileMeta{ + Path: fileName, + Type: mydump.SourceTypeCSV, + FileSize: int64(len(s)), + }, + }) + } + tblMetas = append(tblMetas, &mydump.MDTableMeta{ + DB: db, + Name: tbl.Name, + DataFiles: fileInfos, + }) + } + dbMetas = append(dbMetas, &mydump.MDDatabaseMeta{ + Name: db, + Tables: tblMetas, + }) + } + + err := rc.checkCSVHeader(ctx, dbMetas) + c.Assert(err, IsNil) + if ca.level != passed { + c.Assert(rc.checkTemplate.FailedCount(ca.level), Equals, 1) + } + } + +} diff --git a/br/pkg/lightning/restore/restore.go b/br/pkg/lightning/restore/restore.go index 0358bc17045e5..bf417602bf4d2 100644 --- a/br/pkg/lightning/restore/restore.go +++ b/br/pkg/lightning/restore/restore.go @@ -870,7 +870,7 @@ func verifyLocalFile(ctx context.Context, cpdb checkpoints.DB, dir string) error for tableName, engineIDs := range targetTables { for _, engineID := range engineIDs { _, eID := backend.MakeUUID(tableName, engineID) - file := local.File{UUID: eID} + file := local.Engine{UUID: eID} err := file.Exist(dir) if err != nil { log.L().Error("can't find local file", @@ -1490,7 +1490,7 @@ func (rc *Controller) restoreTables(ctx context.Context) error { if err != nil { return errors.Trace(err) } - tr, err := NewTableRestore(tableName, tableMeta, dbInfo, tableInfo, cp, igCols.Columns) + tr, err := NewTableRestore(tableName, tableMeta, dbInfo, tableInfo, cp, igCols.ColumnsMap()) if err != nil { return errors.Trace(err) } @@ -1968,6 +1968,11 @@ func (rc *Controller) DataCheck(ctx context.Context) error { } } } + err = rc.checkCSVHeader(ctx, rc.dbMetas) + if err != nil { + return err + } + if len(checkPointCriticalMsgs) != 0 { rc.checkTemplate.Collect(Critical, false, strings.Join(checkPointCriticalMsgs, "\n")) } else { @@ -2301,6 +2306,16 @@ func (cr *chunkRestore) encodeLoop( pauser, maxKvPairsCnt := rc.pauser, rc.cfg.TikvImporter.MaxKVPairs initializedColumns, reachEOF := false, false + // filteredColumns is column names that excluded ignored columns + // WARN: this might be not correct when different SQL statements contains different fields, + // but since ColumnPermutation also depends on the hypothesis that the columns in one source file is the same + // so this should be ok. + var filteredColumns []string + ignoreColumns, err1 := rc.cfg.Mydumper.IgnoreColumns.GetIgnoreColumns(t.dbInfo.Name, t.tableInfo.Core.Name.O, rc.cfg.Mydumper.CaseSensitive) + if err1 != nil { + err = err1 + return + } for !reachEOF { if err = pauser.Wait(ctx); err != nil { return @@ -2331,6 +2346,26 @@ func (cr *chunkRestore) encodeLoop( return } } + filteredColumns = columnNames + if ignoreColumns != nil && len(ignoreColumns.Columns) > 0 { + filteredColumns = make([]string, 0, len(columnNames)) + ignoreColsMap := ignoreColumns.ColumnsMap() + if len(columnNames) > 0 { + for _, c := range columnNames { + if _, ok := ignoreColsMap[c]; !ok { + filteredColumns = append(filteredColumns, c) + } + } + } else { + // init column names by table schema + // after filtered out some columns, we must explicitly set the columns for TiDB backend + for _, col := range t.tableInfo.Core.Columns { + if _, ok := ignoreColsMap[col.Name.L]; !col.Hidden && !ok { + filteredColumns = append(filteredColumns, col.Name.O) + } + } + } + } initializedColumns = true } case io.EOF: @@ -2364,7 +2399,7 @@ func (cr *chunkRestore) encodeLoop( continue } - kvPacket = append(kvPacket, deliveredKVs{kvs: kvs, columns: columnNames, offset: newOffset, rowID: rowID}) + kvPacket = append(kvPacket, deliveredKVs{kvs: kvs, columns: filteredColumns, offset: newOffset, rowID: rowID}) kvSize += kvs.Size() failpoint.Inject("mock-kv-size", func(val failpoint.Value) { kvSize += uint64(val.(int)) diff --git a/br/pkg/lightning/restore/restore_test.go b/br/pkg/lightning/restore/restore_test.go index baf3881457a6c..19c68b9db287f 100644 --- a/br/pkg/lightning/restore/restore_test.go +++ b/br/pkg/lightning/restore/restore_test.go @@ -745,30 +745,124 @@ func (s *tableRestoreSuite) TestGetColumnsNames(c *C) { func (s *tableRestoreSuite) TestInitializeColumns(c *C) { ccp := &checkpoints.ChunkCheckpoint{} - c.Assert(s.tr.initializeColumns(nil, ccp), IsNil) - c.Assert(ccp.ColumnPermutation, DeepEquals, []int{0, 1, 2, -1}) - ccp.ColumnPermutation = nil - c.Assert(s.tr.initializeColumns([]string{"b", "c", "a"}, ccp), IsNil) - c.Assert(ccp.ColumnPermutation, DeepEquals, []int{2, 0, 1, -1}) + defer func() { + s.tr.ignoreColumns = nil + }() + + cases := []struct { + columns []string + ignoreColumns map[string]struct{} + expectedPermutation []int + errPat string + }{ + { + nil, + nil, + []int{0, 1, 2, -1}, + "", + }, + { + nil, + map[string]struct{}{"b": {}}, + []int{0, -1, 2, -1}, + "", + }, + { + []string{"b", "c", "a"}, + nil, + []int{2, 0, 1, -1}, + "", + }, + { + []string{"b", "c", "a"}, + map[string]struct{}{"b": {}}, + []int{2, -1, 1, -1}, + "", + }, + { + []string{"b"}, + nil, + []int{-1, 0, -1, -1}, + "", + }, + { + []string{"_tidb_rowid", "b", "a", "c"}, + nil, + []int{2, 1, 3, 0}, + "", + }, + { + []string{"_tidb_rowid", "b", "a", "c"}, + map[string]struct{}{"b": {}, "_tidb_rowid": {}}, + []int{2, -1, 3, -1}, + "", + }, + { + []string{"_tidb_rowid", "b", "a", "c", "d"}, + nil, + nil, + `unknown columns in header \[d\]`, + }, + { + []string{"e", "b", "c", "d"}, + nil, + nil, + `unknown columns in header \[e d\]`, + }, + } + + for _, testCase := range cases { + ccp.ColumnPermutation = nil + s.tr.ignoreColumns = testCase.ignoreColumns + err := s.tr.initializeColumns(testCase.columns, ccp) + if len(testCase.errPat) > 0 { + c.Assert(err, NotNil) + c.Assert(err, ErrorMatches, testCase.errPat) + } else { + c.Assert(ccp.ColumnPermutation, DeepEquals, testCase.expectedPermutation) + } + } +} - ccp.ColumnPermutation = nil - c.Assert(s.tr.initializeColumns([]string{"b"}, ccp), IsNil) - c.Assert(ccp.ColumnPermutation, DeepEquals, []int{-1, 0, -1, -1}) +func (s *tableRestoreSuite) TestInitializeColumnsGenerated(c *C) { + p := parser.New() + p.SetSQLMode(mysql.ModeANSIQuotes) + se := tmock.NewContext() - ccp.ColumnPermutation = nil - c.Assert(s.tr.initializeColumns([]string{"_tidb_rowid", "b", "a", "c"}, ccp), IsNil) - c.Assert(ccp.ColumnPermutation, DeepEquals, []int{2, 1, 3, 0}) + cases := []struct { + schema string + columns []string + expectedPermutation []int + }{ + { + "CREATE TABLE `table` (a INT, b INT, C INT, d INT AS (a * 2))", + []string{"b", "c", "a"}, + []int{2, 0, 1, -1, -1}, + }, + // all generated columns and none input columns + { + "CREATE TABLE `table` (a bigint as (1 + 2) stored, b text as (sha1(repeat('x', a))) stored)", + []string{}, + []int{-1, -1, -1}, + }, + } - ccp.ColumnPermutation = nil - err := s.tr.initializeColumns([]string{"_tidb_rowid", "b", "a", "c", "d"}, ccp) - c.Assert(err, NotNil) - c.Assert(err, ErrorMatches, `unknown columns in header \[d\]`) + for _, testCase := range cases { + node, err := p.ParseOneStmt(testCase.schema, "", "") + c.Assert(err, IsNil) + core, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 0xabcdef) + c.Assert(err, IsNil) + core.State = model.StatePublic + tableInfo := &checkpoints.TidbTableInfo{Name: "table", DB: "db", Core: core} + s.tr, err = NewTableRestore("`db`.`table`", s.tableMeta, s.dbInfo, tableInfo, &checkpoints.TableCheckpoint{}, nil) + c.Assert(err, IsNil) + ccp := &checkpoints.ChunkCheckpoint{} - ccp.ColumnPermutation = nil - err = s.tr.initializeColumns([]string{"e", "b", "c", "d"}, ccp) - c.Assert(err, NotNil) - c.Assert(err, ErrorMatches, `unknown columns in header \[e d\]`) + err = s.tr.initializeColumns(testCase.columns, ccp) + c.Assert(err, IsNil) + c.Assert(ccp.ColumnPermutation, DeepEquals, testCase.expectedPermutation) + } } func (s *tableRestoreSuite) TestCompareChecksumSuccess(c *C) { @@ -1234,6 +1328,7 @@ func (s *chunkRestoreSuite) TestEncodeLoop(c *C) { c.Assert(kvs, HasLen, 1) c.Assert(kvs[0].rowID, Equals, int64(19)) c.Assert(kvs[0].offset, Equals, int64(36)) + c.Assert(kvs[0].columns, DeepEquals, []string(nil)) kvs = <-kvsCh c.Assert(len(kvs), Equals, 0) @@ -1306,6 +1401,7 @@ func (s *chunkRestoreSuite) TestEncodeLoopDeliverLimit(c *C) { rc := &Controller{pauser: DeliverPauser, cfg: cfg} c.Assert(failpoint.Enable( "github.com/pingcap/tidb/br/pkg/lightning/restore/mock-kv-size", "return(110000000)"), IsNil) + defer failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/restore/mock-kv-size") _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) c.Assert(err, IsNil) @@ -1355,7 +1451,7 @@ func (s *chunkRestoreSuite) TestEncodeLoopDeliverErrored(c *C) { func (s *chunkRestoreSuite) TestEncodeLoopColumnsMismatch(c *C) { dir := c.MkDir() fileName := "db.table.000.csv" - err := os.WriteFile(filepath.Join(dir, fileName), []byte("1,2,3,4\r\n4,5,6,7\r\n"), 0o644) + err := os.WriteFile(filepath.Join(dir, fileName), []byte("1,2\r\n4,5,6,7\r\n"), 0o644) c.Assert(err, IsNil) store, err := storage.NewLocalStorage(dir) @@ -1385,12 +1481,117 @@ func (s *chunkRestoreSuite) TestEncodeLoopColumnsMismatch(c *C) { Timestamp: 1234567895, }) c.Assert(err, IsNil) + defer kvEncoder.Close() _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) - c.Assert(err, ErrorMatches, "in file db.table.2.sql:0 at offset 8: column count mismatch, expected 3, got 4") + c.Assert(err, ErrorMatches, "in file db.table.2.sql:0 at offset 4: column count mismatch, expected 3, got 2") c.Assert(kvsCh, HasLen, 0) } +func (s *chunkRestoreSuite) TestEncodeLoopIgnoreColumnsCSV(c *C) { + log.InitLogger(&log.Config{}, "error") + cases := []struct { + s string + ignoreColumns []*config.IgnoreColumns + kvs deliveredKVs + header bool + }{ + { + "1,2,3\r\n4,5,6\r\n", + []*config.IgnoreColumns{ + { + DB: "db", + Table: "table", + Columns: []string{"a"}, + }, + }, + deliveredKVs{ + rowID: 1, + offset: 6, + columns: []string{"b", "c"}, + }, + false, + }, + { + "b,c\r\n2,3\r\n5,6\r\n", + []*config.IgnoreColumns{ + { + TableFilter: []string{"db*.tab*"}, + Columns: []string{"b"}, + }, + }, + deliveredKVs{ + rowID: 1, + offset: 9, + columns: []string{"c"}, + }, + true, + }, + } + + for _, cs := range cases { + // reset test + s.SetUpTest(c) + s.testEncodeLoopIgnoreColumnsCSV(c, cs.s, cs.ignoreColumns, cs.kvs, cs.header) + } +} + +func (s *chunkRestoreSuite) testEncodeLoopIgnoreColumnsCSV( + c *C, + f string, + ignoreColumns []*config.IgnoreColumns, + deliverKV deliveredKVs, + header bool, +) { + dir := c.MkDir() + fileName := "db.table.000.csv" + err := os.WriteFile(filepath.Join(dir, fileName), []byte(f), 0o644) + c.Assert(err, IsNil) + + store, err := storage.NewLocalStorage(dir) + c.Assert(err, IsNil) + + ctx := context.Background() + cfg := config.NewConfig() + cfg.Mydumper.IgnoreColumns = ignoreColumns + cfg.Mydumper.CSV.Header = header + rc := &Controller{pauser: DeliverPauser, cfg: cfg} + + reader, err := store.Open(ctx, fileName) + c.Assert(err, IsNil) + w := worker.NewPool(ctx, 5, "io") + p, err := mydump.NewCSVParser(&cfg.Mydumper.CSV, reader, 111, w, cfg.Mydumper.CSV.Header, nil) + c.Assert(err, IsNil) + + err = s.cr.parser.Close() + c.Assert(err, IsNil) + s.cr.parser = p + + kvsCh := make(chan []deliveredKVs, 2) + deliverCompleteCh := make(chan deliverResult) + kvEncoder, err := tidb.NewTiDBBackend(nil, config.ReplaceOnDup, errormanager.New(nil, config.NewConfig())).NewEncoder( + s.tr.encTable, + &kv.SessionOptions{ + SQLMode: s.cfg.TiDB.SQLMode, + Timestamp: 1234567895, + }) + c.Assert(err, IsNil) + defer kvEncoder.Close() + + _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) + c.Assert(err, IsNil) + c.Assert(kvsCh, HasLen, 2) + + kvs := <-kvsCh + c.Assert(kvs, HasLen, 2) + c.Assert(kvs[0].rowID, Equals, deliverKV.rowID) + c.Assert(kvs[0].offset, Equals, deliverKV.offset) + c.Assert(kvs[0].columns, DeepEquals, deliverKV.columns) + + kvs = <-kvsCh + c.Assert(len(kvs), Equals, 0) +} + func (s *chunkRestoreSuite) TestRestore(c *C) { ctx := context.Background() diff --git a/br/pkg/lightning/restore/table_restore.go b/br/pkg/lightning/restore/table_restore.go index 8da5a210ce885..8664943e75199 100644 --- a/br/pkg/lightning/restore/table_restore.go +++ b/br/pkg/lightning/restore/table_restore.go @@ -53,7 +53,7 @@ type TableRestore struct { alloc autoid.Allocators logger log.Logger - ignoreColumns []string + ignoreColumns map[string]struct{} } func NewTableRestore( @@ -62,7 +62,7 @@ func NewTableRestore( dbInfo *checkpoints.TidbDBInfo, tableInfo *checkpoints.TidbTableInfo, cp *checkpoints.TableCheckpoint, - ignoreColumns []string, + ignoreColumns map[string]struct{}, ) (*TableRestore, error) { idAlloc := kv.NewPanickingAllocators(cp.AllocBase) tbl, err := tables.TableFromMeta(idAlloc, tableInfo.Core) @@ -167,15 +167,21 @@ func (tr *TableRestore) initializeColumns(columns []string, ccp *checkpoints.Chu return nil } -func createColumnPermutation(columns []string, ignoreColumns []string, tableInfo *model.TableInfo) ([]int, error) { +func createColumnPermutation(columns []string, ignoreColumns map[string]struct{}, tableInfo *model.TableInfo) ([]int, error) { var colPerm []int if len(columns) == 0 { colPerm = make([]int, 0, len(tableInfo.Columns)+1) shouldIncludeRowID := common.TableHasAutoRowID(tableInfo) // no provided columns, so use identity permutation. - for i := range tableInfo.Columns { - colPerm = append(colPerm, i) + for i, col := range tableInfo.Columns { + idx := i + if _, ok := ignoreColumns[col.Name.L]; ok { + idx = -1 + } else if col.IsGenerated() { + idx = -1 + } + colPerm = append(colPerm, idx) } if shouldIncludeRowID { colPerm = append(colPerm, -1) @@ -839,7 +845,7 @@ func (tr *TableRestore) postProcess( return true, nil } -func parseColumnPermutations(tableInfo *model.TableInfo, columns []string, ignoreColumns []string) ([]int, error) { +func parseColumnPermutations(tableInfo *model.TableInfo, columns []string, ignoreColumns map[string]struct{}) ([]int, error) { colPerm := make([]int, 0, len(tableInfo.Columns)+1) columnMap := make(map[string]int) @@ -847,13 +853,6 @@ func parseColumnPermutations(tableInfo *model.TableInfo, columns []string, ignor columnMap[column] = i } - ignoreMap := make(map[string]int) - for _, column := range ignoreColumns { - if i, ok := columnMap[column]; ok { - ignoreMap[column] = i - } - } - tableColumnMap := make(map[string]int) for i, col := range tableInfo.Columns { tableColumnMap[col.Name.L] = i @@ -863,7 +862,7 @@ func parseColumnPermutations(tableInfo *model.TableInfo, columns []string, ignor var unknownCols []string for _, c := range columns { if _, ok := tableColumnMap[c]; !ok && c != model.ExtraHandleName.L { - if _, ignore := ignoreMap[c]; !ignore { + if _, ignore := ignoreColumns[c]; !ignore { unknownCols = append(unknownCols, c) } } @@ -875,7 +874,7 @@ func parseColumnPermutations(tableInfo *model.TableInfo, columns []string, ignor for _, colInfo := range tableInfo.Columns { if i, ok := columnMap[colInfo.Name.L]; ok { - if _, ignore := ignoreMap[colInfo.Name.L]; !ignore { + if _, ignore := ignoreColumns[colInfo.Name.L]; !ignore { colPerm = append(colPerm, i) } else { log.L().Debug("column ignored by user requirements", @@ -896,11 +895,16 @@ func parseColumnPermutations(tableInfo *model.TableInfo, columns []string, ignor colPerm = append(colPerm, -1) } } + // append _tidb_rowid column + rowIDIdx := -1 if i, ok := columnMap[model.ExtraHandleName.L]; ok { - colPerm = append(colPerm, i) - } else if common.TableHasAutoRowID(tableInfo) { - colPerm = append(colPerm, -1) + if _, ignored := ignoreColumns[model.ExtraHandleName.L]; !ignored { + rowIDIdx = i + } } + // FIXME: the schema info for tidb backend is not complete, so always add the _tidb_rowid field. + // Other logic should ignore this extra field if not needed. + colPerm = append(colPerm, rowIDIdx) return colPerm, nil } diff --git a/br/pkg/mock/mock_cluster.go b/br/pkg/mock/mock_cluster.go index c60f4764b4ffc..e00246835c365 100644 --- a/br/pkg/mock/mock_cluster.go +++ b/br/pkg/mock/mock_cluster.go @@ -39,46 +39,49 @@ type Cluster struct { kv.Storage *server.TiDBDriver *domain.Domain - DSN string - PDClient pd.Client + DSN string + PDClient pd.Client + HttpServer *http.Server } // NewCluster create a new mock cluster. func NewCluster() (*Cluster, error) { + cluster := &Cluster{} + pprofOnce.Do(func() { go func() { // Make sure pprof is registered. _ = pprof.Handler addr := "0.0.0.0:12235" log.Info("start pprof", zap.String("addr", addr)) - if e := http.ListenAndServe(addr, nil); e != nil { + cluster.HttpServer = &http.Server{Addr: addr} + if e := cluster.HttpServer.ListenAndServe(); e != nil { log.Warn("fail to start pprof", zap.String("addr", addr), zap.Error(e)) } }() }) - var mockCluster testutils.Cluster storage, err := mockstore.NewMockStore( mockstore.WithClusterInspector(func(c testutils.Cluster) { mockstore.BootstrapWithSingleStore(c) - mockCluster = c + cluster.Cluster = c }), ) if err != nil { return nil, errors.Trace(err) } + cluster.Storage = storage + session.SetSchemaLease(0) session.DisableStats4Test() dom, err := session.BootstrapSession(storage) if err != nil { return nil, errors.Trace(err) } - return &Cluster{ - Storage: storage, - Cluster: mockCluster, - Domain: dom, - PDClient: storage.(tikv.Storage).GetRegionCache().PDClient(), - }, nil + cluster.Domain = dom + + cluster.PDClient = storage.(tikv.Storage).GetRegionCache().PDClient() + return cluster, nil } // Start runs a mock cluster. @@ -130,11 +133,14 @@ func (mock *Cluster) Stop() { mock.Domain.Close() } if mock.Storage != nil { - mock.Storage.Close() + _ = mock.Storage.Close() } if mock.Server != nil { mock.Server.Close() } + if mock.HttpServer != nil { + _ = mock.HttpServer.Close() + } } type configOverrider func(*mysql.Config) diff --git a/br/pkg/mock/mock_cluster_test.go b/br/pkg/mock/mock_cluster_test.go index bee7220613f2d..01dde0c158901 100644 --- a/br/pkg/mock/mock_cluster_test.go +++ b/br/pkg/mock/mock_cluster_test.go @@ -5,32 +5,19 @@ package mock_test import ( "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/br/pkg/mock" - "github.com/pingcap/tidb/util/testleak" + "github.com/stretchr/testify/require" + "go.uber.org/goleak" ) -func Test(t *testing.T) { - TestingT(t) -} - -var _ = Suite(&testClusterSuite{}) - -type testClusterSuite struct { - mock *mock.Cluster -} - -func (s *testClusterSuite) SetUpSuite(c *C) { - var err error - s.mock, err = mock.NewCluster() - c.Assert(err, IsNil) -} - -func (s *testClusterSuite) TearDownSuite(c *C) { - testleak.AfterTest(c)() -} - -func (s *testClusterSuite) TestSmoke(c *C) { - c.Assert(s.mock.Start(), IsNil) - s.mock.Stop() +func TestSmoke(t *testing.T) { + defer goleak.VerifyNone( + t, + goleak.IgnoreTopFunction("github.com/klauspost/compress/zstd.(*blockDec).startDecoder"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start")) + m, err := mock.NewCluster() + require.NoError(t, err) + require.NoError(t, m.Start()) + m.Stop() } diff --git a/br/pkg/redact/redact_test.go b/br/pkg/redact/redact_test.go index 9ee743292519a..5647a681974f3 100644 --- a/br/pkg/redact/redact_test.go +++ b/br/pkg/redact/redact_test.go @@ -3,29 +3,24 @@ package redact_test import ( + "encoding/hex" "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/br/pkg/redact" + "github.com/stretchr/testify/require" + "go.uber.org/goleak" ) -type testRedactSuite struct{} +func TestRedact(t *testing.T) { + defer goleak.VerifyNone(t) -func (s *testRedactSuite) SetUpSuite(c *C) {} -func (s *testRedactSuite) TearDownSuite(c *C) {} - -var _ = Suite(&testRedactSuite{}) - -func TestT(t *testing.T) {} - -func (s *testRedactSuite) TestRedact(c *C) { redacted, secret := "?", "secret" redact.InitRedact(false) - c.Assert(redact.String(secret), Equals, secret) - c.Assert(redact.Key([]byte(secret)), Equals, secret) + require.Equal(t, redact.String(secret), secret) + require.Equal(t, redact.Key([]byte(secret)), hex.EncodeToString([]byte(secret))) redact.InitRedact(true) - c.Assert(redact.String(secret), Equals, redacted) - c.Assert(redact.Key([]byte(secret)), Equals, redacted) + require.Equal(t, redact.String(secret), redacted) + require.Equal(t, redact.Key([]byte(secret)), redacted) } diff --git a/br/pkg/utils/backoff_test.go b/br/pkg/utils/backoff_test.go index 31b1a0214d255..e8af459c78de6 100644 --- a/br/pkg/utils/backoff_test.go +++ b/br/pkg/utils/backoff_test.go @@ -4,35 +4,20 @@ package utils_test import ( "context" + "testing" "time" - . "github.com/pingcap/check" berrors "github.com/pingcap/tidb/br/pkg/errors" - "github.com/pingcap/tidb/br/pkg/mock" "github.com/pingcap/tidb/br/pkg/utils" - "github.com/pingcap/tidb/util/testleak" + "github.com/stretchr/testify/require" "go.uber.org/multierr" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" ) -var _ = Suite(&testBackofferSuite{}) +func TestBackoffWithSuccess(t *testing.T) { + t.Parallel() -type testBackofferSuite struct { - mock *mock.Cluster -} - -func (s *testBackofferSuite) SetUpSuite(c *C) { - var err error - s.mock, err = mock.NewCluster() - c.Assert(err, IsNil) -} - -func (s *testBackofferSuite) TearDownSuite(c *C) { - testleak.AfterTest(c)() -} - -func (s *testBackofferSuite) TestBackoffWithSuccess(c *C) { var counter int backoffer := utils.NewBackoffer(10, time.Nanosecond, time.Nanosecond) err := utils.WithRetry(context.Background(), func() error { @@ -47,11 +32,13 @@ func (s *testBackofferSuite) TestBackoffWithSuccess(c *C) { } return nil }, backoffer) - c.Assert(counter, Equals, 3) - c.Assert(err, IsNil) + require.Equal(t, 3, counter) + require.NoError(t, err) } -func (s *testBackofferSuite) TestBackoffWithFatalError(c *C) { +func TestBackoffWithFatalError(t *testing.T) { + t.Parallel() + var counter int backoffer := utils.NewBackoffer(10, time.Nanosecond, time.Nanosecond) gRPCError := status.Error(codes.Unavailable, "transport is closing") @@ -69,16 +56,18 @@ func (s *testBackofferSuite) TestBackoffWithFatalError(c *C) { } return nil }, backoffer) - c.Assert(counter, Equals, 4) - c.Assert(multierr.Errors(err), DeepEquals, []error{ + require.Equal(t, 4, counter) + require.Equal(t, []error{ gRPCError, berrors.ErrKVEpochNotMatch, berrors.ErrKVDownloadFailed, berrors.ErrKVRangeIsEmpty, - }) + }, multierr.Errors(err)) } -func (s *testBackofferSuite) TestBackoffWithFatalRawGRPCError(c *C) { +func TestBackoffWithFatalRawGRPCError(t *testing.T) { + t.Parallel() + var counter int canceledError := status.Error(codes.Canceled, "context canceled") backoffer := utils.NewBackoffer(10, time.Nanosecond, time.Nanosecond) @@ -86,21 +75,21 @@ func (s *testBackofferSuite) TestBackoffWithFatalRawGRPCError(c *C) { defer func() { counter++ }() return canceledError // nolint:wrapcheck }, backoffer) - c.Assert(counter, Equals, 1) - c.Assert(multierr.Errors(err), DeepEquals, []error{ - canceledError, - }) + require.Equal(t, 1, counter) + require.Equal(t, []error{canceledError}, multierr.Errors(err)) } -func (s *testBackofferSuite) TestBackoffWithRetryableError(c *C) { +func TestBackoffWithRetryableError(t *testing.T) { + t.Parallel() + var counter int backoffer := utils.NewBackoffer(10, time.Nanosecond, time.Nanosecond) err := utils.WithRetry(context.Background(), func() error { defer func() { counter++ }() return berrors.ErrKVEpochNotMatch }, backoffer) - c.Assert(counter, Equals, 10) - c.Assert(multierr.Errors(err), DeepEquals, []error{ + require.Equal(t, 10, counter) + require.Equal(t, []error{ berrors.ErrKVEpochNotMatch, berrors.ErrKVEpochNotMatch, berrors.ErrKVEpochNotMatch, @@ -111,10 +100,12 @@ func (s *testBackofferSuite) TestBackoffWithRetryableError(c *C) { berrors.ErrKVEpochNotMatch, berrors.ErrKVEpochNotMatch, berrors.ErrKVEpochNotMatch, - }) + }, multierr.Errors(err)) } -func (s *testBackofferSuite) TestPdBackoffWithRetryableError(c *C) { +func TestPdBackoffWithRetryableError(t *testing.T) { + t.Parallel() + var counter int backoffer := utils.NewPDReqBackoffer() gRPCError := status.Error(codes.Unavailable, "transport is closing") @@ -122,8 +113,8 @@ func (s *testBackofferSuite) TestPdBackoffWithRetryableError(c *C) { defer func() { counter++ }() return gRPCError }, backoffer) - c.Assert(counter, Equals, 16) - c.Assert(multierr.Errors(err), DeepEquals, []error{ + require.Equal(t, 16, counter) + require.Equal(t, []error{ gRPCError, gRPCError, gRPCError, @@ -140,5 +131,5 @@ func (s *testBackofferSuite) TestPdBackoffWithRetryableError(c *C) { gRPCError, gRPCError, gRPCError, - }) + }, multierr.Errors(err)) } diff --git a/br/pkg/utils/env_test.go b/br/pkg/utils/env_test.go index 89ad840ee7fc7..6934ef3c6e870 100644 --- a/br/pkg/utils/env_test.go +++ b/br/pkg/utils/env_test.go @@ -4,15 +4,14 @@ package utils import ( "os" + "testing" - "github.com/pingcap/check" + "github.com/stretchr/testify/require" ) -type envSuit struct{} +func TestProxyFields(t *testing.T) { + t.Parallel() -var _ = check.Suite(&envSuit{}) - -func (s *envSuit) TestProxyFields(c *check.C) { revIndex := map[string]int{ "http_proxy": 0, "https_proxy": 1, @@ -25,20 +24,20 @@ func (s *envSuit) TestProxyFields(c *check.C) { // Each bit of the mask decided whether this index of `envs` would be set. for mask := 0; mask <= 0b111; mask++ { for _, env := range envs { - c.Assert(os.Unsetenv(env), check.IsNil) + require.NoError(t, os.Unsetenv(env)) } for i := 0; i < 3; i++ { if (1< 0x80; +Error 1105: Cannot convert string '80' from binary to utf8mb4 diff --git a/cmd/explaintest/run-tests.sh b/cmd/explaintest/run-tests.sh index 1d8061f90d0f0..0ebcabca0b2d5 100755 --- a/cmd/explaintest/run-tests.sh +++ b/cmd/explaintest/run-tests.sh @@ -31,6 +31,8 @@ stats="s" set -eu trap 'set +e; PIDS=$(jobs -p); [ -n "$PIDS" ] && kill -9 $PIDS' EXIT +# make tests stable time zone wise +export TZ="Asia/Shanghai" function help_message() { diff --git a/cmd/explaintest/t/explain_generate_column_substitute.test b/cmd/explaintest/t/explain_generate_column_substitute.test index 47b18596b942a..06a546a05c6e5 100644 --- a/cmd/explaintest/t/explain_generate_column_substitute.test +++ b/cmd/explaintest/t/explain_generate_column_substitute.test @@ -234,3 +234,21 @@ create index eidx on t02 ((lower(a))); select * from t02 use index(eidx) where lower(a) < 'c'; select @@tidb_allow_function_for_expression_index; + +CREATE TABLE `PK_S_MULTI_30_tmp` ( + `COL1` double NOT NULL, + `COL2` double NOT NULL, + `COL3` double DEFAULT NULL, + PRIMARY KEY (`COL1`,`COL2`) /*T![clustered_index] NONCLUSTERED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; +insert into `PK_S_MULTI_30_tmp` values (-1.7976931348623157e308, 0, 0); +alter table `PK_S_MULTI_30_tmp` add index ((ceil(col1))); +alter table `PK_S_MULTI_30_tmp` add index ((floor(col1))); +insert into `PK_S_MULTI_30_tmp` values (-1.7976931348623157e308, 1, 0); +select * from `PK_S_MULTI_30_tmp` use index (expression_index) where ceil(col1) = ceil(-1.7976931348623157e+308); +select * from `PK_S_MULTI_30_tmp` ignore index (expression_index) where ceil(col1) = ceil(-1.7976931348623157e+308); +select * from `PK_S_MULTI_30_tmp` use index (expression_index) where floor(col1) = floor(-1.7976931348623157e+308); +select * from `PK_S_MULTI_30_tmp` ignore index (expression_index) where floor(col1) = floor(-1.7976931348623157e+308); +alter table PK_S_MULTI_30_tmp add index eidx ((truncate(col1, 3))); +select * from PK_S_MULTI_30_tmp ignore index (eidx) where truncate(col1, 3) <= truncate(-1.7976931348623157e308, 3); +select * from PK_S_MULTI_30_tmp use index (eidx) where truncate(col1, 3) <= truncate(-1.7976931348623157e308, 3); diff --git a/cmd/explaintest/t/new_character_set_builtin.test b/cmd/explaintest/t/new_character_set_builtin.test index 0c6fefac7919d..ff4c2a9354f74 100644 --- a/cmd/explaintest/t/new_character_set_builtin.test +++ b/cmd/explaintest/t/new_character_set_builtin.test @@ -152,3 +152,71 @@ select hex(aes_decrypt(0x91C44DE866D0745252, '1234567890123456', '12345678901234 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; + +-- test for builtin function ord() +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 ord(a), ord(b), ord(c) from t; +set @@tidb_enable_vectorized_expression = true; +select ord(a), ord(b), ord(c) from t; +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 ('一', '一', 0xe4b880); +insert into t values ('一', '一', 0xd2bb); +insert into t values ('一', '一', 0xe4ba8c); +insert into t values ('一', '一', 0xb6fe); + +set @@tidb_enable_vectorized_expression = true; +select hex(concat(a, c)), hex(concat(b, c)) from t; +select hex(concat(a, 0xe4b880)), hex(concat(b, 0xd2bb)) from t; +select a = 0xe4b880, b = 0xd2bb from t; +select a = c, b = c from t; +select hex(insert(a, 1, 2, 0xe4ba8c)), hex(insert(b, 1, 2, 0xb6fe)) from t; +select hex(insert(a, 1, 2, c)), hex(insert(b, 1, 2, c)) from t; +select hex(lpad(a, 5, 0xe4ba8c)), hex(lpad(b, 5, 0xb6fe)) from t; +select hex(lpad(a, 5, c)), hex(lpad(b, 5, c)) from t; +select hex(rpad(a, 5, 0xe4ba8c)), hex(rpad(b, 5, 0xb6fe)) from t; +select hex(rpad(a, 5, c)), hex(rpad(b, 5, c)) from t; +select hex(elt(2, a, 0xe4ba8c)), hex(elt(2, b, 0xb6fe)) from t; +select hex(elt(2, a, c)), hex(elt(2, b, c)) from t; +select hex(instr(a, 0xe4b880)), hex(instr(b, 0xd2bb)) from t; +select hex(position(a in 0xe4b880)), hex(position(b in 0xd2bb)) from t; +select a like 0xe4b880, b like 0xd2bb from t; + +--error ER_CANNOT_CONVERT_STRING +select a = 0xb6fe from t; +--error ER_CANNOT_CONVERT_STRING +select b = 0xe4ba8c from t; +--error ER_CANNOT_CONVERT_STRING +select concat(a, 0xb6fe) from t; +--error ER_CANNOT_CONVERT_STRING +select concat(b, 0xe4ba8c) from t; + +set @@tidb_enable_vectorized_expression = false; +select hex(concat(a, c)), hex(concat(b, c)) from t; +select hex(concat(a, 0xe4b880)), hex(concat(b, 0xd2bb)) from t; +select a = 0xe4b880, b = 0xd2bb from t; +select a = c, b = c from t; +select hex(insert(a, 1, 2, 0xe4ba8c)), hex(insert(b, 1, 2, 0xb6fe)) from t; +select hex(insert(a, 1, 2, c)), hex(insert(b, 1, 2, c)) from t; +select hex(lpad(a, 5, 0xe4ba8c)), hex(lpad(b, 5, 0xb6fe)) from t; +select hex(lpad(a, 5, c)), hex(lpad(b, 5, c)) from t; +select hex(rpad(a, 5, 0xe4ba8c)), hex(rpad(b, 5, 0xb6fe)) from t; +select hex(rpad(a, 5, c)), hex(rpad(b, 5, c)) from t; +select hex(elt(2, a, 0xe4ba8c)), hex(elt(2, b, 0xb6fe)) from t; +select hex(elt(2, a, c)), hex(elt(2, b, c)) from t; +select hex(instr(a, 0xe4b880)), hex(instr(b, 0xd2bb)) from t; +select hex(position(a in 0xe4b880)), hex(position(b in 0xd2bb)) from t; +select a like 0xe4b880, b like 0xd2bb from t; + +--error ER_CANNOT_CONVERT_STRING +select a = 0xb6fe from t; +--error ER_CANNOT_CONVERT_STRING +select b = 0xe4ba8c from t; +--error ER_CANNOT_CONVERT_STRING +select concat(a, 0xb6fe) from t; +--error ER_CANNOT_CONVERT_STRING +select concat(b, 0xe4ba8c) from t; diff --git a/cmd/explaintest/t/new_character_set_invalid.test b/cmd/explaintest/t/new_character_set_invalid.test new file mode 100644 index 0000000000000..34031d0b83ef8 --- /dev/null +++ b/cmd/explaintest/t/new_character_set_invalid.test @@ -0,0 +1,17 @@ +set @@sql_mode = 'strict_trans_tables'; +drop table if exists t; +create table t (a varchar(255) charset gbk, b varchar(255) charset ascii, c varchar(255) charset utf8); +insert into t values ('中文', 'asdf', '字符集'); +-- error 1366: Incorrect string value '\xC3\x80' for column 'a' +insert into t values ('À', 'ø', '😂'); +-- error 1366: Incorrect string value '\xC3\x80\xE4\xB8\xAD\xE6...' for column 'a' +insert into t values ('中文À中文', 'asdføfdsa', '字符集😂字符集'); +-- error 1366: Incorrect string value '\xFF\xFF' for column 'a' +insert into t values (0x4040ffff, 0x4040ffff, 0x4040ffff); +select * from t; + +set @@sql_mode = ''; +insert into t values ('À', 'ø', '😂'); +insert into t values ('中文À中文', 'asdføfdsa', '字符集😂字符集'); +insert into t values (0x4040ffff, 0x4040ffff, 0x4040ffff); +select * from t; diff --git a/cmd/explaintest/t/select.test b/cmd/explaintest/t/select.test index 77ace4a6d3512..c53aca600e2b9 100644 --- a/cmd/explaintest/t/select.test +++ b/cmd/explaintest/t/select.test @@ -244,3 +244,8 @@ insert into precise_types values ( 18446744073709551614 ); SELECT a, b, c, d FROM precise_types; + +create table t3(a char(10), primary key (a)); +insert into t3 values ('a'); +--error ER_CANNOT_CONVERT_STRING +select * from t3 where a > 0x80; diff --git a/ddl/backfilling.go b/ddl/backfilling.go index 6be728a56640c..7c6bfad46c938 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -425,6 +425,13 @@ func (w *worker) handleReorgTasks(reorgInfo *reorgInfo, totalAddedCount *int64, } func tryDecodeToHandleString(key kv.Key) string { + defer func() { + if r := recover(); r != nil { + logutil.BgLogger().Warn("tryDecodeToHandleString panic", + zap.Any("recover()", r), + zap.Binary("key", key)) + } + }() handle, err := tablecodec.DecodeRowKey(key) if err != nil { recordPrefixIdx := bytes.Index(key, []byte("_r")) diff --git a/ddl/column_change_test.go b/ddl/column_change_test.go index 67e3927860346..c2c5c8282f2a8 100644 --- a/ddl/column_change_test.go +++ b/ddl/column_change_test.go @@ -48,17 +48,18 @@ type testColumnChangeSuite struct { func (s *testColumnChangeSuite) SetUpSuite(c *C) { SetWaitTimeWhenErrorOccurred(1 * time.Microsecond) s.store = testCreateStore(c, "test_column_change") - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) }() - s.dbInfo = testSchemaInfo(c, d, "test_index_change") + s.dbInfo, err = testSchemaInfo(d, "test_index_change") + c.Assert(err, IsNil) testCreateSchema(c, testNewContext(d), d, s.dbInfo) } @@ -67,12 +68,12 @@ func (s *testColumnChangeSuite) TearDownSuite(c *C) { } func (s *testColumnChangeSuite) TestColumnChange(c *C) { - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) @@ -80,7 +81,7 @@ func (s *testColumnChangeSuite) TestColumnChange(c *C) { // create table t (c1 int, c2 int); tblInfo := testTableInfo(c, d, "t", 2) ctx := testNewContext(d) - err := ctx.NewTxn(context.Background()) + err = ctx.NewTxn(context.Background()) c.Assert(err, IsNil) testCreateTable(c, ctx, d, s.dbInfo, tblInfo) // insert t values (1, 2); @@ -163,12 +164,12 @@ func (s *testColumnChangeSuite) TestColumnChange(c *C) { } func (s *testColumnChangeSuite) TestModifyAutoRandColumnWithMetaKeyChanged(c *C) { - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) diff --git a/ddl/column_test.go b/ddl/column_test.go index acf98bb26fc87..8ea1e1b5cb35f 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -46,14 +46,15 @@ type testColumnSuite struct { func (s *testColumnSuite) SetUpSuite(c *C) { s.store = testCreateStore(c, "test_column") - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) - s.dbInfo = testSchemaInfo(c, d, "test_column") + s.dbInfo, err = testSchemaInfo(d, "test_column") + c.Assert(err, IsNil) testCreateSchema(c, testNewContext(d), d, s.dbInfo) c.Assert(d.Stop(), IsNil) } @@ -185,12 +186,12 @@ func testDropColumns(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, } func (s *testColumnSuite) TestColumnBasic(c *C) { - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) @@ -208,7 +209,7 @@ func (s *testColumnSuite) TestColumnBasic(c *C) { c.Assert(err, IsNil) } - err := ctx.NewTxn(context.Background()) + err = ctx.NewTxn(context.Background()) c.Assert(err, IsNil) i := int64(0) @@ -836,16 +837,16 @@ func (s *testColumnSuite) testGetColumn(t table.Table, name string, isExist bool } func (s *testColumnSuite) TestAddColumn(c *C) { - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) tblInfo := testTableInfo(c, d, "t", 3) ctx := testNewContext(d) - err := ctx.NewTxn(context.Background()) + err = ctx.NewTxn(context.Background()) c.Assert(err, IsNil) testCreateTable(c, ctx, d, s.dbInfo, tblInfo) @@ -924,16 +925,16 @@ func (s *testColumnSuite) TestAddColumn(c *C) { } func (s *testColumnSuite) TestAddColumns(c *C) { - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) tblInfo := testTableInfo(c, d, "t", 3) ctx := testNewContext(d) - err := ctx.NewTxn(context.Background()) + err = ctx.NewTxn(context.Background()) c.Assert(err, IsNil) testCreateTable(c, ctx, d, s.dbInfo, tblInfo) @@ -1009,16 +1010,16 @@ func (s *testColumnSuite) TestAddColumns(c *C) { } func (s *testColumnSuite) TestDropColumn(c *C) { - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) tblInfo := testTableInfo(c, d, "t2", 4) ctx := testNewContext(d) - err := ctx.NewTxn(context.Background()) + err = ctx.NewTxn(context.Background()) c.Assert(err, IsNil) testCreateTable(c, ctx, d, s.dbInfo, tblInfo) @@ -1085,16 +1086,16 @@ func (s *testColumnSuite) TestDropColumn(c *C) { } func (s *testColumnSuite) TestDropColumns(c *C) { - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) tblInfo := testTableInfo(c, d, "t2", 4) ctx := testNewContext(d) - err := ctx.NewTxn(context.Background()) + err = ctx.NewTxn(context.Background()) c.Assert(err, IsNil) testCreateTable(c, ctx, d, s.dbInfo, tblInfo) @@ -1154,12 +1155,12 @@ func (s *testColumnSuite) TestDropColumns(c *C) { } func (s *testColumnSuite) TestModifyColumn(c *C) { - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) ctx := testNewContext(d) defer func() { @@ -1222,12 +1223,12 @@ func (s *testColumnSuite) TestFieldCase(c *C) { } func (s *testColumnSuite) TestAutoConvertBlobTypeByLength(c *C) { - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) // Close the customized ddl(worker goroutine included) after the test is finished, otherwise, it will // cause go routine in TiDB leak test. defer func() { diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index da26fed509461..af48b5a0a1900 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -548,13 +548,13 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromStringToOthers(c *C) tk.MustExec("insert into t values ('123.45', '123.45', '123.45', '123.45', '123.45', '123.45', '123', '123')") tk.MustExec("alter table t modify c decimal(7, 4)") tk.MustExec("alter table t modify vc decimal(7, 4)") - tk.MustExec("alter table t modify bny decimal(7, 4)") + tk.MustGetErrCode("alter table t modify bny decimal(7, 4)", mysql.ErrTruncatedWrongValue) tk.MustExec("alter table t modify vbny decimal(7, 4)") tk.MustExec("alter table t modify bb decimal(7, 4)") tk.MustExec("alter table t modify txt decimal(7, 4)") tk.MustExec("alter table t modify e decimal(7, 4)") tk.MustExec("alter table t modify s decimal(7, 4)") - tk.MustQuery("select * from t").Check(testkit.Rows("123.4500 123.4500 123.4500 123.4500 123.4500 123.4500 1.0000 1.0000")) + tk.MustQuery("select * from t").Check(testkit.Rows("123.4500 123.4500 123.45\x00\x00 123.4500 123.4500 123.4500 1.0000 1.0000")) // double reset(tk) tk.MustExec("insert into t values ('123.45', '123.45', '123.45', '123.45', '123.45', '123.45', '123', '123')") @@ -2227,3 +2227,14 @@ func (s *testColumnTypeChangeSuite) TestChangeFromBitToStringInvalidUtf8ErrMsg(c errMsg := "[table:1366]Incorrect string value '\\xEC\\xBD' for column 'a'" tk.MustGetErrMsg("alter table t modify column a varchar(31) collate utf8mb4_general_ci;", errMsg) } + +func (s *testColumnTypeChangeSuite) TestForIssue24621(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char(250));") + tk.MustExec("insert into t values('0123456789abc');") + errMsg := "[types:1265]Data truncated for column 'a', value is '0123456789abc'" + tk.MustGetErrMsg("alter table t modify a char(12) null;", errMsg) +} diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 74e81ba6374c8..3ae2259ddaf07 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -1819,6 +1819,8 @@ func (s *serialTestStateChangeSuite) TestCreateExpressionIndex(c *C) { stateWriteOnlySQLs := []string{"insert into t values (8, 8)", "begin pessimistic;", "insert into t select * from t", "rollback", "insert into t set b = 9", "update t set b = 7 where a = 2", "delete from t where b = 3"} stateWriteReorganizationSQLs := []string{"insert into t values (10, 10)", "begin pessimistic;", "insert into t select * from t", "rollback", "insert into t set b = 11", "update t set b = 7 where a = 5", "delete from t where b = 6"} + // If waitReorg timeout, the worker may enter writeReorg more than 2 times. + reorgTime := 0 var checkErr error d := s.dom.DDL() originalCallback := d.GetHook() @@ -1848,6 +1850,11 @@ func (s *serialTestStateChangeSuite) TestCreateExpressionIndex(c *C) { } // (1, 7), (2, 7), (5, 5), (0, 6), (8, 8), (0, 9) case model.StateWriteReorganization: + if reorgTime < 2 { + reorgTime++ + } else { + return + } for _, sql := range stateWriteReorganizationSQLs { _, checkErr = tk1.Exec(sql) if checkErr != nil { @@ -1880,6 +1887,8 @@ func (s *serialTestStateChangeSuite) TestCreateUniqueExpressionIndex(c *C) { stateDeleteOnlySQLs := []string{"insert into t values (5, 5)", "begin pessimistic;", "insert into t select * from t", "rollback", "insert into t set b = 6", "update t set b = 7 where a = 1", "delete from t where b = 4"} + // If waitReorg timeout, the worker may enter writeReorg more than 2 times. + reorgTime := 0 var checkErr error d := s.dom.DDL() originalCallback := d.GetHook() @@ -1932,6 +1941,11 @@ func (s *serialTestStateChangeSuite) TestCreateUniqueExpressionIndex(c *C) { } // (1, 7), (2, 7), (5, 5), (0, 6), (8, 8), (0, 9) case model.StateWriteReorganization: + if reorgTime < 2 { + reorgTime++ + } else { + return + } _, checkErr = tk1.Exec("insert into t values (10, 10) on duplicate key update a = 11") if checkErr != nil { return diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 3c69c44083487..60e8f8b8f67cc 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -3619,3 +3619,115 @@ func (s *testIntegrationSuite3) TestIssue29282(c *C) { c.Fail() } } + +func (s *testIntegrationSuite3) TestIssue29326(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1 (id int)") + tk.MustExec("insert into t1 values(1)") + defer tk.MustExec("drop table t1") + + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t2 (id int)") + tk.MustExec("insert into t2 values(1)") + defer tk.MustExec("drop table t2") + + tk.MustExec("drop view if exists v1") + defer tk.MustExec("drop view if exists v1") + + tk.MustExec("create view v1 as select 1,1") + rs, err := tk.Exec("select * from v1") + c.Assert(err, IsNil) + tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 1")) + c.Assert(rs.Fields()[0].Column.Name.O, Equals, "1") + c.Assert(rs.Fields()[1].Column.Name.O, Equals, "Name_exp_1") + + tk.MustExec("drop view if exists v1") + tk.MustExec("create view v1 as select 1, 2, 1, 2, 1, 2, 1, 2") + rs, err = tk.Exec("select * from v1") + c.Assert(err, IsNil) + tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 2 1 2 1 2 1 2")) + c.Assert(rs.Fields()[0].Column.Name.O, Equals, "1") + c.Assert(rs.Fields()[1].Column.Name.O, Equals, "2") + c.Assert(rs.Fields()[2].Column.Name.O, Equals, "Name_exp_1") + c.Assert(rs.Fields()[3].Column.Name.O, Equals, "Name_exp_2") + c.Assert(rs.Fields()[4].Column.Name.O, Equals, "Name_exp_1_1") + c.Assert(rs.Fields()[5].Column.Name.O, Equals, "Name_exp_1_2") + c.Assert(rs.Fields()[6].Column.Name.O, Equals, "Name_exp_2_1") + c.Assert(rs.Fields()[7].Column.Name.O, Equals, "Name_exp_2_2") + + tk.MustExec("drop view if exists v1") + tk.MustExec("create view v1 as select 't', 't', 1 as t") + rs, err = tk.Exec("select * from v1") + c.Assert(err, IsNil) + tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("t t 1")) + c.Assert(rs.Fields()[0].Column.Name.O, Equals, "Name_exp_t") + c.Assert(rs.Fields()[1].Column.Name.O, Equals, "Name_exp_1_t") + c.Assert(rs.Fields()[2].Column.Name.O, Equals, "t") + + tk.MustExec("drop view if exists v1") + tk.MustExec("create definer=`root`@`127.0.0.1` view v1 as select 1, 1 union all select 1, 1") + tk.MustQuery("show create view v1").Check(testkit.Rows("v1 CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` " + + "SQL SECURITY DEFINER VIEW `v1` (`1`, `Name_exp_1`) " + + "AS SELECT 1 AS `1`,1 AS `Name_exp_1` UNION ALL SELECT 1 AS `1`,1 AS `1` " + + "utf8mb4 utf8mb4_bin")) + rs, err = tk.Exec("select * from v1") + c.Assert(err, IsNil) + tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 1", "1 1")) + c.Assert(rs.Fields()[0].Column.Name.O, Equals, "1") + c.Assert(rs.Fields()[1].Column.Name.O, Equals, "Name_exp_1") + + tk.MustExec("drop view if exists v1") + tk.MustExec("create definer=`root`@`127.0.0.1` view v1 as select 'id', id from t1") + tk.MustQuery("show create view v1").Check(testkit.Rows("v1 CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` " + + "SQL SECURITY DEFINER VIEW `v1` (`Name_exp_id`, `id`) " + + "AS SELECT _UTF8MB4'id' AS `Name_exp_id`,`id` AS `id` FROM `test`.`t1` " + + "utf8mb4 utf8mb4_bin")) + rs, err = tk.Exec("select * from v1") + c.Assert(err, IsNil) + tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("id 1")) + c.Assert(rs.Fields()[0].Column.Name.O, Equals, "Name_exp_id") + c.Assert(rs.Fields()[1].Column.Name.O, Equals, "id") + + tk.MustExec("drop view if exists v1") + tk.MustExec("create definer=`root`@`127.0.0.1` view v1 as select 1, (select id from t1 where t1.id=t2.id) as '1' from t2") + tk.MustQuery("show create view v1").Check(testkit.Rows("v1 CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` " + + "SQL SECURITY DEFINER VIEW `v1` (`Name_exp_1`, `1`) " + + "AS SELECT 1 AS `Name_exp_1`,(SELECT `id` AS `id` FROM `test`.`t1` WHERE `t1`.`id`=`t2`.`id`) AS `1` FROM `test`.`t2` " + + "utf8mb4 utf8mb4_bin")) + rs, err = tk.Exec("select * from v1") + c.Assert(err, IsNil) + tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 1")) + c.Assert(rs.Fields()[0].Column.Name.O, Equals, "Name_exp_1") + c.Assert(rs.Fields()[1].Column.Name.O, Equals, "1") + + tk.MustExec("drop view if exists v1") + tk.MustExec("create definer=`root`@`127.0.0.1` view v1 as select 1 as 'abs(t1.id)', abs(t1.id) from t1") + tk.MustQuery("show create view v1").Check(testkit.Rows("v1 CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` " + + "SQL SECURITY DEFINER VIEW `v1` (`abs(t1.id)`, `Name_exp_abs(t1.id)`) " + + "AS SELECT 1 AS `abs(t1.id)`,ABS(`t1`.`id`) AS `Name_exp_abs(t1.id)` FROM `test`.`t1` " + + "utf8mb4 utf8mb4_bin")) + rs, err = tk.Exec("select * from v1") + c.Assert(err, IsNil) + tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 1")) + c.Assert(rs.Fields()[0].Column.Name.O, Equals, "abs(t1.id)") + c.Assert(rs.Fields()[1].Column.Name.O, Equals, "Name_exp_abs(t1.id)") + + tk.MustExec("drop view if exists v1") + err = tk.ExecToErr("create definer=`root`@`127.0.0.1` view v1 as select 1 as t,1 as t") + c.Assert(infoschema.ErrColumnExists.Equal(err), IsTrue) + + tk.MustExec("drop view if exists v1") + err = tk.ExecToErr("create definer=`root`@`127.0.0.1` view v1 as select 1 as id, id from t1") + c.Assert(infoschema.ErrColumnExists.Equal(err), IsTrue) + + tk.MustExec("drop view if exists v1") + err = tk.ExecToErr("create definer=`root`@`127.0.0.1` view v1 as select * from t1 left join t2 on t1.id=t2.id") + c.Assert(infoschema.ErrColumnExists.Equal(err), IsTrue) + + tk.MustExec("drop view if exists v1") + err = tk.ExecToErr("create definer=`root`@`127.0.0.1` view v1 as select t1.id, t2.id from t1,t2 where t1.id=t2.id") + c.Assert(infoschema.ErrColumnExists.Equal(err), IsTrue) +} diff --git a/ddl/db_test.go b/ddl/db_test.go index cb1c90b54df49..173ffa2679e4b 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -5664,6 +5664,30 @@ func (s *testSerialDBSuite) TestSetTableFlashReplica(c *C) { c.Assert(err.Error(), Equals, "the tiflash replica count: 2 should be less than the total tiflash server count: 0") } +func (s *testSerialDBSuite) TestForbitCacheTableForSystemTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + sysTables := make([]string, 0, 24) + memOrSysDB := []string{"MySQL", "INFORMATION_SCHEMA", "PERFORMANCE_SCHEMA", "METRICS_SCHEMA"} + for _, db := range memOrSysDB { + tk.MustExec("use " + db) + tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil) + rows := tk.MustQuery("show tables").Rows() + for i := 0; i < len(rows); i++ { + sysTables = append(sysTables, rows[i][0].(string)) + } + for _, one := range sysTables { + _, err := tk.Exec(fmt.Sprintf("alter table `%s` cache", one)) + if db == "MySQL" { + c.Assert(err.Error(), Equals, "[ddl:8200]ALTER table cache for tables in system database is currently unsupported") + } else { + c.Assert(err.Error(), Equals, fmt.Sprintf("[planner:1142]ALTER command denied to user 'root'@'%%' for table '%s'", strings.ToLower(one))) + } + + } + sysTables = sysTables[:0] + } +} + func (s *testSerialDBSuite) TestSetTableFlashReplicaForSystemTable(c *C) { tk := testkit.NewTestKit(c, s.store) sysTables := make([]string, 0, 24) @@ -5944,6 +5968,36 @@ func (s *testSerialDBSuite) TestSkipSchemaChecker(c *C) { c.Assert(terror.ErrorEqual(domain.ErrInfoSchemaChanged, err), IsTrue) } +// See issue: https://github.com/pingcap/tidb/issues/29752 +// Ref https://dev.mysql.com/doc/refman/8.0/en/rename-table.html +func (s *testDBSuite2) TestRenameTableWithLocked(c *C) { + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.EnableTableLock = true + }) + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database renamedb") + tk.MustExec("create database renamedb2") + tk.MustExec("use renamedb") + tk.MustExec("DROP TABLE IF EXISTS t1;") + tk.MustExec("CREATE TABLE t1 (a int);") + + tk.MustExec("LOCK TABLES t1 WRITE;") + tk.MustGetErrCode("drop database renamedb2;", errno.ErrLockOrActiveTransaction) + tk.MustExec("RENAME TABLE t1 TO t2;") + tk.MustQuery("select * from renamedb.t2").Check(testkit.Rows()) + tk.MustExec("UNLOCK TABLES") + tk.MustExec("RENAME TABLE t2 TO t1;") + tk.MustQuery("select * from renamedb.t1").Check(testkit.Rows()) + + tk.MustExec("LOCK TABLES t1 READ;") + tk.MustGetErrCode("RENAME TABLE t1 TO t2;", errno.ErrTableNotLockedForWrite) + tk.MustExec("UNLOCK TABLES") + + tk.MustExec("drop database renamedb") +} + func (s *testDBSuite2) TestLockTables(c *C) { if israce.RaceEnabled { c.Skip("skip race test") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 76d9a786988cd..fd12090bc8b19 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -6606,7 +6606,10 @@ func (d *ddl) AlterTableCache(ctx sessionctx.Context, ti ast.Ident) (err error) return nil } - if t.Meta().TempTableType != model.TempTableNone { + // forbit cache table in system database. + if util.IsMemOrSysDB(schema.Name.L) { + return errors.Trace(errUnsupportedAlterCacheForSysTable) + } else if t.Meta().TempTableType != model.TempTableNone { return errors.Trace(ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache")) } diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index 9ac70a5ccb124..f158c3203abd3 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -96,16 +96,14 @@ func TestT(t *testing.T) { testleak.AfterTestT(t)() } -func testNewDDLAndStart(ctx context.Context, c *C, options ...Option) *ddl { +func testNewDDLAndStart(ctx context.Context, options ...Option) (*ddl, error) { // init infoCache and a stub infoSchema ic := infoschema.NewCache(2) ic.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0), 0) options = append(options, WithInfoCache(ic)) d := newDDL(ctx, options...) err := d.Start(nil) - c.Assert(err, IsNil) - - return d + return d, err } func testCreateStore(c *C, name string) kv.Storage { diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 90fb4604650a7..4d327d5b2e699 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -62,12 +62,12 @@ func (s *testDDLSuite) TestCheckOwner(c *C) { c.Assert(err, IsNil) }() - d1 := testNewDDLAndStart( + d1, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d1.Stop() c.Assert(err, IsNil) @@ -95,12 +95,12 @@ func (s *testDDLSuite) TestNotifyDDLJob(c *C) { } } - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) @@ -139,12 +139,12 @@ func (s *testDDLSuite) TestNotifyDDLJob(c *C) { // Test the notification mechanism that the owner and the server receiving the DDL request are not on the same TiDB. // And the etcd client is nil. - d1 := testNewDDLAndStart( + d1, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d1.Stop() c.Assert(err, IsNil) @@ -178,12 +178,12 @@ func (s *testDDLSerialSuite) testRunWorker(c *C) { }() RunWorker = false - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) testCheckOwner(c, d, false) defer func() { err := d.Stop() @@ -195,12 +195,12 @@ func (s *testDDLSerialSuite) testRunWorker(c *C) { c.Assert(worker, IsNil) // Make sure the DDL job can be done and exit that goroutine. RunWorker = true - d1 := testNewDDLAndStart( + d1, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) testCheckOwner(c, d1, true) defer func() { err := d1.Stop() @@ -217,12 +217,12 @@ func (s *testDDLSuite) TestSchemaError(c *C) { c.Assert(err, IsNil) }() - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) @@ -239,12 +239,12 @@ func (s *testDDLSuite) TestTableError(c *C) { c.Assert(err, IsNil) }() - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) @@ -254,14 +254,15 @@ func (s *testDDLSuite) TestTableError(c *C) { // Schema ID is wrong, so dropping table is failed. doDDLJobErr(c, -1, 1, model.ActionDropTable, nil, ctx, d) // Table ID is wrong, so dropping table is failed. - dbInfo := testSchemaInfo(c, d, "test_ddl") + dbInfo, err := testSchemaInfo(d, "test_ddl") + c.Assert(err, IsNil) testCreateSchema(c, testNewContext(d), d, dbInfo) job := doDDLJobErr(c, dbInfo.ID, -1, model.ActionDropTable, nil, ctx, d) // Table ID or schema ID is wrong, so getting table is failed. tblInfo := testTableInfo(c, d, "t", 3) testCreateTable(c, ctx, d, dbInfo, tblInfo) - err := kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { + err = kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { job.SchemaID = -1 job.TableID = -1 t := meta.NewMeta(txn) @@ -291,18 +292,19 @@ func (s *testDDLSuite) TestViewError(c *C) { c.Assert(err, IsNil) }() - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) }() ctx := testNewContext(d) - dbInfo := testSchemaInfo(c, d, "test_ddl") + dbInfo, err := testSchemaInfo(d, "test_ddl") + c.Assert(err, IsNil) testCreateSchema(c, testNewContext(d), d, dbInfo) // Table ID or schema ID is wrong, so getting table is failed. @@ -325,12 +327,12 @@ func (s *testDDLSuite) TestInvalidDDLJob(c *C) { err := store.Close() c.Assert(err, IsNil) }() - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) @@ -344,7 +346,7 @@ func (s *testDDLSuite) TestInvalidDDLJob(c *C) { BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{}, } - err := d.doDDLJob(ctx, job) + err = d.doDDLJob(ctx, job) c.Assert(err.Error(), Equals, "[ddl:8204]invalid ddl job type: none") } @@ -355,12 +357,12 @@ func (s *testDDLSuite) TestForeignKeyError(c *C) { c.Assert(err, IsNil) }() - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) @@ -370,7 +372,8 @@ func (s *testDDLSuite) TestForeignKeyError(c *C) { doDDLJobErr(c, -1, 1, model.ActionAddForeignKey, nil, ctx, d) doDDLJobErr(c, -1, 1, model.ActionDropForeignKey, nil, ctx, d) - dbInfo := testSchemaInfo(c, d, "test_ddl") + dbInfo, err := testSchemaInfo(d, "test_ddl") + c.Assert(err, IsNil) tblInfo := testTableInfo(c, d, "t", 3) testCreateSchema(c, ctx, d, dbInfo) testCreateTable(c, ctx, d, dbInfo, tblInfo) @@ -384,12 +387,12 @@ func (s *testDDLSuite) TestIndexError(c *C) { c.Assert(err, IsNil) }() - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) @@ -400,7 +403,8 @@ func (s *testDDLSuite) TestIndexError(c *C) { doDDLJobErr(c, -1, 1, model.ActionAddIndex, nil, ctx, d) doDDLJobErr(c, -1, 1, model.ActionDropIndex, nil, ctx, d) - dbInfo := testSchemaInfo(c, d, "test_ddl") + dbInfo, err := testSchemaInfo(d, "test_ddl") + c.Assert(err, IsNil) tblInfo := testTableInfo(c, d, "t", 3) testCreateSchema(c, ctx, d, dbInfo) testCreateTable(c, ctx, d, dbInfo, tblInfo) @@ -430,19 +434,20 @@ func (s *testDDLSuite) TestColumnError(c *C) { err := store.Close() c.Assert(err, IsNil) }() - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) }() ctx := testNewContext(d) - dbInfo := testSchemaInfo(c, d, "test_ddl") + dbInfo, err := testSchemaInfo(d, "test_ddl") + c.Assert(err, IsNil) tblInfo := testTableInfo(c, d, "t", 3) testCreateSchema(c, ctx, d, dbInfo) testCreateTable(c, ctx, d, dbInfo, tblInfo) @@ -729,17 +734,18 @@ func (s *testDDLSerialSuite) TestCancelJob(c *C) { err := store.Close() c.Assert(err, IsNil) }() - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) }() - dbInfo := testSchemaInfo(c, d, "test_cancel_job") + dbInfo, err := testSchemaInfo(d, "test_cancel_job") + c.Assert(err, IsNil) testCreateSchema(c, testNewContext(d), d, dbInfo) // create a partition table. partitionTblInfo := testTableInfoWithPartition(c, d, "t_partition", 5) @@ -748,7 +754,7 @@ func (s *testDDLSerialSuite) TestCancelJob(c *C) { // create table t (c1 int, c2 int, c3 int, c4 int, c5 int); tblInfo := testTableInfo(c, d, "t", 5) ctx := testNewContext(d) - err := ctx.NewTxn(context.Background()) + err = ctx.NewTxn(context.Background()) c.Assert(err, IsNil) err = ctx.GetSessionVars().SetSystemVar("tidb_enable_exchange_partition", "1") c.Assert(err, IsNil) @@ -898,7 +904,8 @@ func (s *testDDLSerialSuite) TestCancelJob(c *C) { testCheckTableState(c, d, dbInfo, tblInfo1, model.StateNone) // for create database - dbInfo1 := testSchemaInfo(c, d, "test_cancel_job1") + dbInfo1, err := testSchemaInfo(d, "test_cancel_job1") + c.Assert(err, IsNil) updateTest(&tests[9]) doDDLJobErrWithSchemaState(ctx, d, c, dbInfo1.ID, 0, model.ActionCreateSchema, []interface{}{dbInfo1}, &cancelState) c.Check(checkErr, IsNil) @@ -1438,18 +1445,18 @@ func (s *testDDLSuite) TestParallelDDL(c *C) { err := store.Close() c.Assert(err, IsNil) }() - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) }() ctx := testNewContext(d) - err := ctx.NewTxn(context.Background()) + err = ctx.NewTxn(context.Background()) c.Assert(err, IsNil) /* build structure: @@ -1469,7 +1476,8 @@ func (s *testDDLSuite) TestParallelDDL(c *C) { } */ // create database test_parallel_ddl_1; - dbInfo1 := testSchemaInfo(c, d, "test_parallel_ddl_1") + dbInfo1, err := testSchemaInfo(d, "test_parallel_ddl_1") + c.Assert(err, IsNil) testCreateSchema(c, ctx, d, dbInfo1) // create table t1 (c1 int, c2 int); tblInfo1 := testTableInfo(c, d, "t1", 2) @@ -1494,7 +1502,8 @@ func (s *testDDLSuite) TestParallelDDL(c *C) { _, err = tbl2.AddRecord(ctx, types.MakeDatums(3, 3, 3)) c.Assert(err, IsNil) // create database test_parallel_ddl_2; - dbInfo2 := testSchemaInfo(c, d, "test_parallel_ddl_2") + dbInfo2, err := testSchemaInfo(d, "test_parallel_ddl_2") + c.Assert(err, IsNil) testCreateSchema(c, ctx, d, dbInfo2) // create table t3 (c1 int, c2 int, c3 int, c4 int); tblInfo3 := testTableInfo(c, d, "t3", 4) @@ -1643,12 +1652,12 @@ func (s *testDDLSuite) TestDDLPackageExecuteSQL(c *C) { c.Assert(err, IsNil) }() - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) testCheckOwner(c, d, true) defer func() { err := d.Stop() diff --git a/ddl/error.go b/ddl/error.go index 101e1c9e2f384..b3eeb9655fc77 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -50,6 +50,7 @@ var ( errUnsupportedAlterTableWithoutValidation = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("ALTER TABLE WITHOUT VALIDATION is currently unsupported", nil)) errUnsupportedAlterTableOption = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("This type of ALTER TABLE is currently unsupported", nil)) errUnsupportedAlterReplicaForSysTable = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("ALTER table replica for tables in system database is currently unsupported", nil)) + errUnsupportedAlterCacheForSysTable = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("ALTER table cache for tables in system database is currently unsupported", nil)) errBlobKeyWithoutLength = dbterror.ClassDDL.NewStd(mysql.ErrBlobKeyWithoutLength) errKeyPart0 = dbterror.ClassDDL.NewStd(mysql.ErrKeyPart0) errIncorrectPrefixKey = dbterror.ClassDDL.NewStd(mysql.ErrWrongSubKey) @@ -143,6 +144,8 @@ var ( ErrWrongTableName = dbterror.ClassDDL.NewStd(mysql.ErrWrongTableName) // ErrWrongColumnName returns for wrong column name. ErrWrongColumnName = dbterror.ClassDDL.NewStd(mysql.ErrWrongColumnName) + // ErrWrongUsage returns for wrong ddl syntax usage. + ErrWrongUsage = dbterror.ClassDDL.NewStd(mysql.ErrWrongUsage) // ErrInvalidGroupFuncUse returns for using invalid group functions. ErrInvalidGroupFuncUse = dbterror.ClassDDL.NewStd(mysql.ErrInvalidGroupFuncUse) // ErrTableMustHaveColumns returns for missing column when creating a table. diff --git a/ddl/fail_test.go b/ddl/fail_test.go index ef671a45bf9f0..e5ac586976ae0 100644 --- a/ddl/fail_test.go +++ b/ddl/fail_test.go @@ -25,12 +25,12 @@ import ( ) func (s *testColumnChangeSuite) TestFailBeforeDecodeArgs(c *C) { - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) @@ -38,7 +38,7 @@ func (s *testColumnChangeSuite) TestFailBeforeDecodeArgs(c *C) { // create table t_fail (c1 int, c2 int); tblInfo := testTableInfo(c, d, "t_fail", 2) ctx := testNewContext(d) - err := ctx.NewTxn(context.Background()) + err = ctx.NewTxn(context.Background()) c.Assert(err, IsNil) testCreateTable(c, ctx, d, s.dbInfo, tblInfo) // insert t_fail values (1, 2); diff --git a/ddl/foreign_key_test.go b/ddl/foreign_key_test.go index 699205380e53d..1d7bd4d345102 100644 --- a/ddl/foreign_key_test.go +++ b/ddl/foreign_key_test.go @@ -112,24 +112,25 @@ func getForeignKey(t table.Table, name string) *model.FKInfo { } func (s *testForeignKeySuite) TestForeignKey(c *C) { - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) }() s.d = d - s.dbInfo = testSchemaInfo(c, d, "test_foreign") + s.dbInfo, err = testSchemaInfo(d, "test_foreign") + c.Assert(err, IsNil) ctx := testNewContext(d) s.ctx = ctx testCreateSchema(c, ctx, d, s.dbInfo) tblInfo := testTableInfo(c, d, "t", 3) - err := ctx.NewTxn(context.Background()) + err = ctx.NewTxn(context.Background()) c.Assert(err, IsNil) testCreateTable(c, ctx, d, s.dbInfo, tblInfo) diff --git a/ddl/index_change_test.go b/ddl/index_change_test.go index 402ea37d61780..d4e416e307dfa 100644 --- a/ddl/index_change_test.go +++ b/ddl/index_change_test.go @@ -38,17 +38,18 @@ type testIndexChangeSuite struct { func (s *testIndexChangeSuite) SetUpSuite(c *C) { s.store = testCreateStore(c, "test_index_change") - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) }() - s.dbInfo = testSchemaInfo(c, d, "test_index_change") + s.dbInfo, err = testSchemaInfo(d, "test_index_change") + c.Assert(err, IsNil) testCreateSchema(c, testNewContext(d), d, s.dbInfo) } @@ -57,12 +58,12 @@ func (s *testIndexChangeSuite) TearDownSuite(c *C) { } func (s *testIndexChangeSuite) TestIndexChange(c *C) { - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) @@ -72,7 +73,7 @@ func (s *testIndexChangeSuite) TestIndexChange(c *C) { tblInfo.Columns[0].Flag = mysql.PriKeyFlag | mysql.NotNullFlag tblInfo.PKIsHandle = true ctx := testNewContext(d) - err := ctx.NewTxn(context.Background()) + err = ctx.NewTxn(context.Background()) c.Assert(err, IsNil) testCreateTable(c, ctx, d, s.dbInfo, tblInfo) originTable := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID) diff --git a/ddl/label/attributes.go b/ddl/label/attributes.go index 60333426a8b55..b7169411fa6e2 100644 --- a/ddl/label/attributes.go +++ b/ddl/label/attributes.go @@ -125,7 +125,8 @@ func (labels *Labels) Restore() string { // Add will add a new attribute, with validation of all attributes. func (labels *Labels) Add(label Label) error { - for _, l := range *labels { + for i := range *labels { + l := (*labels)[i] res := label.CompatibleWith(&l) if res == AttributesCompatible { continue diff --git a/ddl/partition.go b/ddl/partition.go index 1cf46f6cb91e7..9b13710e87709 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -223,15 +223,19 @@ func updateAddingPartitionInfo(partitionInfo *model.PartitionInfo, tblInfo *mode } // rollbackAddingPartitionInfo remove the `addingDefinitions` in the tableInfo. -func rollbackAddingPartitionInfo(tblInfo *model.TableInfo) ([]int64, []string) { +func rollbackAddingPartitionInfo(tblInfo *model.TableInfo) ([]int64, []string, []*placement.Bundle) { physicalTableIDs := make([]int64, 0, len(tblInfo.Partition.AddingDefinitions)) partNames := make([]string, 0, len(tblInfo.Partition.AddingDefinitions)) + rollbackBundles := make([]*placement.Bundle, 0, len(tblInfo.Partition.AddingDefinitions)) for _, one := range tblInfo.Partition.AddingDefinitions { physicalTableIDs = append(physicalTableIDs, one.ID) partNames = append(partNames, one.Name.L) + if one.PlacementPolicyRef != nil || one.DirectPlacementOpts != nil { + rollbackBundles = append(rollbackBundles, placement.NewBundle(one.ID)) + } } tblInfo.Partition.AddingDefinitions = nil - return physicalTableIDs, partNames + return physicalTableIDs, partNames, rollbackBundles } // checkAddPartitionValue values less than value must be strictly increasing for each partition. @@ -1009,18 +1013,6 @@ func getTableInfoWithDroppingPartitions(t *model.TableInfo) *model.TableInfo { return nt } -func dropRuleBundles(d *ddlCtx, physicalTableIDs []int64) error { - bundles := make([]*placement.Bundle, 0, len(physicalTableIDs)) - for _, ID := range physicalTableIDs { - oldBundle, ok := d.infoCache.GetLatest().BundleByName(placement.GroupID(ID)) - if ok && !oldBundle.IsEmpty() { - bundles = append(bundles, placement.NewBundle(ID)) - } - } - err := infosync.PutRuleBundles(context.TODO(), bundles) - return err -} - func dropLabelRules(d *ddlCtx, schemaName, tableName string, partNames []string) error { deleteRules := make([]string, 0, len(partNames)) for _, partName := range partNames { @@ -1042,12 +1034,10 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( if err != nil { return ver, errors.Trace(err) } - var physicalTableIDs []int64 if job.Type == model.ActionAddTablePartition { - var pNames []string // It is rollbacked from adding table partition, just remove addingDefinitions from tableInfo. - physicalTableIDs, pNames = rollbackAddingPartitionInfo(tblInfo) - err = dropRuleBundles(d, physicalTableIDs) + physicalTableIDs, pNames, rollbackBundles := rollbackAddingPartitionInfo(tblInfo) + err = infosync.PutRuleBundles(context.TODO(), rollbackBundles) if err != nil { job.State = model.JobStateCancelled return ver, errors.Wrapf(err, "failed to notify PD the placement rules") @@ -1066,6 +1056,7 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( return ver, nil } + var physicalTableIDs []int64 if job.State == model.JobStateRunning && job.SchemaState == model.StateNone { // Manually set first state. job.SchemaState = model.StatePublic @@ -1082,11 +1073,6 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( return ver, errors.Trace(err) } physicalTableIDs = updateDroppingPartitionInfo(tblInfo, partNames) - err = dropRuleBundles(d, physicalTableIDs) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Wrapf(err, "failed to notify PD the placement rules") - } err = dropLabelRules(d, job.SchemaName, tblInfo.Name.L, partNames) if err != nil { job.State = model.JobStateCancelled diff --git a/ddl/partition_test.go b/ddl/partition_test.go index 6e84bfdda84cd..f2cdd1c4e3597 100644 --- a/ddl/partition_test.go +++ b/ddl/partition_test.go @@ -41,17 +41,18 @@ func (s *testPartitionSuite) TearDownSuite(c *C) { } func (s *testPartitionSuite) TestDropAndTruncatePartition(c *C) { - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) }() - dbInfo := testSchemaInfo(c, d, "test_partition") + dbInfo, err := testSchemaInfo(d, "test_partition") + c.Assert(err, IsNil) testCreateSchema(c, testNewContext(d), d, dbInfo) // generate 5 partition in tableInfo. tblInfo, partIDs := buildTableInfoWithPartition(c, d) diff --git a/ddl/placement/constraints.go b/ddl/placement/constraints.go index 87619a8df32df..38b678857eee3 100644 --- a/ddl/placement/constraints.go +++ b/ddl/placement/constraints.go @@ -84,7 +84,8 @@ func (constraints *Constraints) Restore() (string, error) { func (constraints *Constraints) Add(label Constraint) error { pass := true - for _, cnst := range *constraints { + for i := range *constraints { + cnst := (*constraints)[i] res := label.CompatibleWith(&cnst) if res == ConstraintCompatible { continue diff --git a/ddl/placement_policy_ddl_test.go b/ddl/placement_policy_ddl_test.go index c713570124efd..64a1b5e0b7ae9 100644 --- a/ddl/placement_policy_ddl_test.go +++ b/ddl/placement_policy_ddl_test.go @@ -60,14 +60,17 @@ func (s *testDDLSuite) TestPlacementPolicyInUse(c *C) { }() ctx := context.Background() - d := testNewDDLAndStart(ctx, c, WithStore(store)) + d, err := testNewDDLAndStart(ctx, WithStore(store)) + c.Assert(err, IsNil) sctx := testNewContext(d) - db1 := testSchemaInfo(c, d, "db1") + db1, err := testSchemaInfo(d, "db1") + c.Assert(err, IsNil) testCreateSchema(c, sctx, d, db1) db1.State = model.StatePublic - db2 := testSchemaInfo(c, d, "db2") + db2, err := testSchemaInfo(d, "db2") + c.Assert(err, IsNil) testCreateSchema(c, sctx, d, db2) db2.State = model.StatePublic @@ -101,7 +104,8 @@ func (s *testDDLSuite) TestPlacementPolicyInUse(c *C) { t3.State = model.StatePublic db1.Tables = append(db1.Tables, t3) - dbP := testSchemaInfo(c, d, "db_p") + dbP, err := testSchemaInfo(d, "db_p") + c.Assert(err, IsNil) dbP.PlacementPolicyRef = &model.PolicyRefInfo{ID: p4.ID, Name: p4.Name} dbP.State = model.StatePublic testCreateSchema(c, sctx, d, dbP) @@ -112,7 +116,7 @@ func (s *testDDLSuite) TestPlacementPolicyInUse(c *C) { t4.State = model.StatePublic db1.Tables = append(db1.Tables, t4) - builder, err := infoschema.NewBuilder(store).InitWithDBInfos( + builder, err := infoschema.NewBuilder(store, nil).InitWithDBInfos( []*model.DBInfo{db1, db2, dbP}, nil, []*model.PolicyInfo{p1, p2, p3, p4, p5}, diff --git a/ddl/placement_policy_test.go b/ddl/placement_policy_test.go index 65b5e7994f713..546b0bb5c7023 100644 --- a/ddl/placement_policy_test.go +++ b/ddl/placement_policy_test.go @@ -17,21 +17,37 @@ package ddl_test import ( "context" "fmt" + "math" "strconv" . "github.com/pingcap/check" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/domain/infosync" mysql "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/store/gcworker" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" ) +func clearAllBundles(c *C) { + bundles, err := infosync.GetAllRuleBundles(context.TODO()) + c.Assert(err, IsNil) + clearBundles := make([]*placement.Bundle, 0, len(bundles)) + for _, bundle := range bundles { + clearBundles = append(clearBundles, &placement.Bundle{ID: bundle.ID}) + } + err = infosync.PutRuleBundles(context.TODO(), clearBundles) + c.Assert(err, IsNil) +} + func (s *testDBSuite6) TestPlacementPolicy(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -330,6 +346,24 @@ func (s *testDBSuite6) TestResetSchemaPlacement(c *C) { tk.MustExec("drop database TestResetPlacementDB;") } +func (s *testDBSuite6) TestCreateOrReplacePlacementPolicy(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop placement policy if exists x") + + // If the policy does not exist, CREATE OR REPLACE PLACEMENT POLICY is the same as CREATE PLACEMENT POLICY + tk.MustExec("create or replace placement policy x primary_region=\"cn-east-1\" regions=\"cn-east-1,cn-east\"") + defer tk.MustExec("drop placement policy if exists x") + tk.MustQuery("show create placement policy x").Check(testkit.Rows("x CREATE PLACEMENT POLICY `x` PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east\"")) + + // If the policy does exist, CREATE OR REPLACE PLACEMENT_POLICY is the same as ALTER PLACEMENT POLICY. + tk.MustExec("create or replace placement policy x primary_region=\"cn-east-1\" regions=\"cn-east-1\"") + tk.MustQuery("show create placement policy x").Check(testkit.Rows("x CREATE PLACEMENT POLICY `x` PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1\"")) + + // Cannot be used together with the if not exists clause. Ref: https://mariadb.com/kb/en/create-view + tk.MustGetErrMsg("create or replace placement policy if not exists x primary_region=\"cn-east-1\" regions=\"cn-east-1\"", "[ddl:1221]Incorrect usage of OR REPLACE and IF NOT EXISTS") +} + func (s *testDBSuite6) TestAlterPlacementPolicy(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -974,6 +1008,103 @@ func (s *testDBSuite6) TestDatabasePlacement(c *C) { )) } +func (s *testDBSuite6) TestDropDatabaseGCPlacement(c *C) { + clearAllBundles(c) + failpoint.Enable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed", `return`) + defer func(originGC bool) { + failpoint.Disable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed") + if originGC { + ddl.EmulatorGCEnable() + } else { + ddl.EmulatorGCDisable() + } + }(ddl.IsEmulatorGCEnable()) + ddl.EmulatorGCDisable() + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("drop database if exists db2") + + tk.MustExec("use test") + + tk.MustExec("create table t (id int) primary_region='r0' regions='r0'") + defer tk.MustExec("drop table if exists t") + + tk.MustExec("create database db2") + tk.MustExec("create table db2.t0 (id int)") + tk.MustExec("create table db2.t1 (id int) primary_region='r1' regions='r1,r2'") + tk.MustExec(`create table db2.t2 (id int) primary_region='r1' regions='r1,r2' PARTITION BY RANGE (id) ( + PARTITION p0 VALUES LESS THAN (100) primary_region='r2' regions='r2', + PARTITION p1 VALUES LESS THAN (1000) + )`) + + is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) + t, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + + tk.MustExec("drop database db2") + + bundles, err := infosync.GetAllRuleBundles(context.TODO()) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 4) + + gcWorker, err := gcworker.NewMockGCWorker(s.store) + c.Assert(err, IsNil) + c.Assert(gcWorker.DeleteRanges(context.TODO(), math.MaxInt64), IsNil) + + bundles, err = infosync.GetAllRuleBundles(context.TODO()) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 1) + c.Assert(bundles[0].ID, Equals, placement.GroupID(t.Meta().ID)) +} + +func (s *testDBSuite6) TestDropTableGCPlacement(c *C) { + clearAllBundles(c) + failpoint.Enable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed", `return`) + defer func(originGC bool) { + failpoint.Disable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed") + if originGC { + ddl.EmulatorGCEnable() + } else { + ddl.EmulatorGCDisable() + } + }(ddl.IsEmulatorGCEnable()) + ddl.EmulatorGCDisable() + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + tk.MustExec("create table t0 (id int)") + defer tk.MustExec("drop table if exists t0") + + tk.MustExec("create table t1 (id int) primary_region='r1' regions='r1,r2'") + defer tk.MustExec("drop table if exists t1") + + tk.MustExec(`create table t2 (id int) primary_region='r1' regions='r1,r2' PARTITION BY RANGE (id) ( + PARTITION p0 VALUES LESS THAN (100) primary_region='r2' regions='r2', + PARTITION p1 VALUES LESS THAN (1000) + )`) + defer tk.MustExec("drop table if exists t2") + + is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) + t1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) + c.Assert(err, IsNil) + + tk.MustExec("drop table t2") + + bundles, err := infosync.GetAllRuleBundles(context.TODO()) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 3) + + gcWorker, err := gcworker.NewMockGCWorker(s.store) + c.Assert(err, IsNil) + c.Assert(gcWorker.DeleteRanges(context.TODO(), math.MaxInt64), IsNil) + + bundles, err = infosync.GetAllRuleBundles(context.TODO()) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 1) + c.Assert(bundles[0].ID, Equals, placement.GroupID(t1.Meta().ID)) +} + func (s *testDBSuite6) TestAlterTablePlacement(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -1061,6 +1192,67 @@ func (s *testDBSuite6) TestAlterTablePlacement(c *C) { ")")) } +func (s *testDBSuite6) TestDropTablePartitionGCPlacement(c *C) { + clearAllBundles(c) + failpoint.Enable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed", `return`) + defer func(originGC bool) { + failpoint.Disable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed") + if originGC { + ddl.EmulatorGCEnable() + } else { + ddl.EmulatorGCDisable() + } + }(ddl.IsEmulatorGCEnable()) + ddl.EmulatorGCDisable() + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + tk.MustExec("create table t0 (id int)") + defer tk.MustExec("drop table if exists t0") + + tk.MustExec("create table t1 (id int) primary_region='r1' regions='r1,r2'") + defer tk.MustExec("drop table if exists t1") + + tk.MustExec(`create table t2 (id int) primary_region='r1' regions='r1,r2' PARTITION BY RANGE (id) ( + PARTITION p0 VALUES LESS THAN (100) primary_region='r2' regions='r2', + PARTITION p1 VALUES LESS THAN (1000) primary_region='r3' regions='r3' + )`) + defer tk.MustExec("drop table if exists t2") + + is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) + t1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) + c.Assert(err, IsNil) + t2, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) + c.Assert(err, IsNil) + + tk.MustExec("alter table t2 drop partition p0") + + bundles, err := infosync.GetAllRuleBundles(context.TODO()) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 4) + + gcWorker, err := gcworker.NewMockGCWorker(s.store) + c.Assert(err, IsNil) + c.Assert(gcWorker.DeleteRanges(context.TODO(), math.MaxInt64), IsNil) + + bundles, err = infosync.GetAllRuleBundles(context.TODO()) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 3) + bundlesMap := make(map[string]*placement.Bundle) + for _, bundle := range bundles { + bundlesMap[bundle.ID] = bundle + } + _, ok := bundlesMap[placement.GroupID(t1.Meta().ID)] + c.Assert(ok, IsTrue) + + _, ok = bundlesMap[placement.GroupID(t2.Meta().ID)] + c.Assert(ok, IsTrue) + + _, ok = bundlesMap[placement.GroupID(t2.Meta().Partition.Definitions[1].ID)] + c.Assert(ok, IsTrue) +} + func (s *testDBSuite6) TestAlterTablePartitionPlacement(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -1322,6 +1514,67 @@ func (s *testDBSuite6) TestTruncateTableWithPlacement(c *C) { } } +func (s *testDBSuite6) TestTruncateTableGCWithPlacement(c *C) { + clearAllBundles(c) + failpoint.Enable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed", `return`) + defer func(originGC bool) { + failpoint.Disable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed") + if originGC { + ddl.EmulatorGCEnable() + } else { + ddl.EmulatorGCDisable() + } + }(ddl.IsEmulatorGCEnable()) + ddl.EmulatorGCDisable() + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + tk.MustExec("create table t0 (id int)") + defer tk.MustExec("drop table if exists t0") + + tk.MustExec("create table t1 (id int) primary_region='r1' regions='r1,r2'") + defer tk.MustExec("drop table if exists t1") + + tk.MustExec(`create table t2 (id int) primary_region='r1' regions='r1,r2' PARTITION BY RANGE (id) ( + PARTITION p0 VALUES LESS THAN (100) primary_region='r2' regions='r2', + PARTITION p1 VALUES LESS THAN (1000) + )`) + defer tk.MustExec("drop table if exists t2") + + tk.MustExec("truncate table t2") + + is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) + t1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) + c.Assert(err, IsNil) + t2, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) + c.Assert(err, IsNil) + + bundles, err := infosync.GetAllRuleBundles(context.TODO()) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 5) + + gcWorker, err := gcworker.NewMockGCWorker(s.store) + c.Assert(err, IsNil) + c.Assert(gcWorker.DeleteRanges(context.TODO(), math.MaxInt64), IsNil) + + bundles, err = infosync.GetAllRuleBundles(context.TODO()) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 3) + bundlesMap := make(map[string]*placement.Bundle) + for _, bundle := range bundles { + bundlesMap[bundle.ID] = bundle + } + _, ok := bundlesMap[placement.GroupID(t1.Meta().ID)] + c.Assert(ok, IsTrue) + + _, ok = bundlesMap[placement.GroupID(t2.Meta().ID)] + c.Assert(ok, IsTrue) + + _, ok = bundlesMap[placement.GroupID(t2.Meta().Partition.Definitions[0].ID)] + c.Assert(ok, IsTrue) +} + func (s *testDBSuite6) TestTruncateTablePartitionWithPlacement(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -1387,6 +1640,67 @@ func (s *testDBSuite6) TestTruncateTablePartitionWithPlacement(c *C) { ")")) } +func (s *testDBSuite6) TestTruncatePartitionGCWithPlacement(c *C) { + clearAllBundles(c) + failpoint.Enable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed", `return`) + defer func(originGC bool) { + failpoint.Disable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed") + if originGC { + ddl.EmulatorGCEnable() + } else { + ddl.EmulatorGCDisable() + } + }(ddl.IsEmulatorGCEnable()) + ddl.EmulatorGCDisable() + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + tk.MustExec("create table t0 (id int)") + defer tk.MustExec("drop table if exists t0") + + tk.MustExec("create table t1 (id int) primary_region='r1' regions='r1,r2'") + defer tk.MustExec("drop table if exists t1") + + tk.MustExec(`create table t2 (id int) primary_region='r1' regions='r1,r2' PARTITION BY RANGE (id) ( + PARTITION p0 VALUES LESS THAN (100) primary_region='r2' regions='r2', + PARTITION p1 VALUES LESS THAN (1000) + )`) + defer tk.MustExec("drop table if exists t2") + + tk.MustExec("alter table t2 truncate partition p0") + + is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) + t1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) + c.Assert(err, IsNil) + t2, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) + c.Assert(err, IsNil) + + bundles, err := infosync.GetAllRuleBundles(context.TODO()) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 4) + + gcWorker, err := gcworker.NewMockGCWorker(s.store) + c.Assert(err, IsNil) + c.Assert(gcWorker.DeleteRanges(context.TODO(), math.MaxInt64), IsNil) + + bundles, err = infosync.GetAllRuleBundles(context.TODO()) + c.Assert(err, IsNil) + c.Assert(len(bundles), Equals, 3) + bundlesMap := make(map[string]*placement.Bundle) + for _, bundle := range bundles { + bundlesMap[bundle.ID] = bundle + } + _, ok := bundlesMap[placement.GroupID(t1.Meta().ID)] + c.Assert(ok, IsTrue) + + _, ok = bundlesMap[placement.GroupID(t2.Meta().ID)] + c.Assert(ok, IsTrue) + + _, ok = bundlesMap[placement.GroupID(t2.Meta().Partition.Definitions[0].ID)] + c.Assert(ok, IsTrue) +} + func (s *testDBSuite6) TestExchangePartitionWithPlacement(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@tidb_enable_exchange_partition=1") diff --git a/ddl/reorg_test.go b/ddl/reorg_test.go index 269df0dc1c76d..4a86b8b649a92 100644 --- a/ddl/reorg_test.go +++ b/ddl/reorg_test.go @@ -41,12 +41,12 @@ func (s *testDDLSuite) TestReorg(c *C) { c.Assert(err, IsNil) }() - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) @@ -60,7 +60,7 @@ func (s *testDDLSuite) TestReorg(c *C) { c.Assert(ctx.Value(testCtxKey), Equals, 1) ctx.ClearValue(testCtxKey) - err := ctx.NewTxn(context.Background()) + err = ctx.NewTxn(context.Background()) c.Assert(err, IsNil) txn, err := ctx.Txn(true) c.Assert(err, IsNil) @@ -192,12 +192,12 @@ func (s *testDDLSuite) TestReorgOwner(c *C) { c.Assert(err, IsNil) }() - d1 := testNewDDLAndStart( + d1, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d1.Stop() c.Assert(err, IsNil) @@ -207,18 +207,19 @@ func (s *testDDLSuite) TestReorgOwner(c *C) { testCheckOwner(c, d1, true) - d2 := testNewDDLAndStart( + d2, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d2.Stop() c.Assert(err, IsNil) }() - dbInfo := testSchemaInfo(c, d1, "test_reorg") + dbInfo, err := testSchemaInfo(d1, "test_reorg") + c.Assert(err, IsNil) testCreateSchema(c, ctx, d1, dbInfo) tblInfo := testTableInfo(c, d1, "t", 3) diff --git a/ddl/restart_test.go b/ddl/restart_test.go index 6248e86806696..db27d3e893ce5 100644 --- a/ddl/restart_test.go +++ b/ddl/restart_test.go @@ -110,12 +110,12 @@ func (s *testSchemaSuite) TestSchemaResume(c *C) { c.Assert(err, IsNil) }() - d1 := testNewDDLAndStart( + d1, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d1.Stop() c.Assert(err, IsNil) @@ -123,7 +123,8 @@ func (s *testSchemaSuite) TestSchemaResume(c *C) { testCheckOwner(c, d1, true) - dbInfo := testSchemaInfo(c, d1, "test_restart") + dbInfo, err := testSchemaInfo(d1, "test_restart") + c.Assert(err, IsNil) job := &model.Job{ SchemaID: dbInfo.ID, Type: model.ActionCreateSchema, @@ -149,18 +150,19 @@ func (s *testStatSuite) TestStat(c *C) { c.Assert(err, IsNil) }() - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) }() - dbInfo := testSchemaInfo(c, d, "test_restart") + dbInfo, err := testSchemaInfo(d, "test_restart") + c.Assert(err, IsNil) testCreateSchema(c, testNewContext(d), d, dbInfo) // TODO: Get this information from etcd. diff --git a/ddl/schema.go b/ddl/schema.go index 718788cc8a524..7604037addd40 100644 --- a/ddl/schema.go +++ b/ddl/schema.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/ddl/label" - "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/meta" @@ -206,20 +205,6 @@ func onDropSchema(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) if err != nil { return ver, errors.Trace(err) } - oldIDs := getIDs(tables) - bundles := make([]*placement.Bundle, 0, len(oldIDs)+1) - for _, ID := range append(oldIDs, dbInfo.ID) { - oldBundle, ok := d.infoCache.GetLatest().BundleByName(placement.GroupID(ID)) - if ok && !oldBundle.IsEmpty() { - bundles = append(bundles, placement.NewBundle(ID)) - } - } - err := infosync.PutRuleBundles(context.TODO(), bundles) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Trace(err) - } - var ruleIDs []string for _, tblInfo := range tables { rules := append(getPartitionRuleIDs(job.SchemaName, tblInfo), fmt.Sprintf(label.TableIDFormat, label.IDPrefix, job.SchemaName, tblInfo.Name.L)) diff --git a/ddl/schema_test.go b/ddl/schema_test.go index 7ddbe1a4ea0db..a5ef31edd389d 100644 --- a/ddl/schema_test.go +++ b/ddl/schema_test.go @@ -39,14 +39,16 @@ func (s *testSchemaSuite) SetUpSuite(c *C) { func (s *testSchemaSuite) TearDownSuite(c *C) { } -func testSchemaInfo(c *C, d *ddl, name string) *model.DBInfo { +func testSchemaInfo(d *ddl, name string) (*model.DBInfo, error) { dbInfo := &model.DBInfo{ Name: model.NewCIStr(name), } genIDs, err := d.genGlobalIDs(1) - c.Assert(err, IsNil) + if err != nil { + return nil, err + } dbInfo.ID = genIDs[0] - return dbInfo + return dbInfo, nil } func testCreateSchema(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo) *model.Job { @@ -129,18 +131,19 @@ func (s *testSchemaSuite) TestSchema(c *C) { err := store.Close() c.Assert(err, IsNil) }() - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) }() ctx := testNewContext(d) - dbInfo := testSchemaInfo(c, d, "test_schema") + dbInfo, err := testSchemaInfo(d, "test_schema") + c.Assert(err, IsNil) // create a database. job := testCreateSchema(c, ctx, d, dbInfo) @@ -181,11 +184,12 @@ func (s *testSchemaSuite) TestSchema(c *C) { Type: model.ActionDropSchema, BinlogInfo: &model.HistoryInfo{}, } - err := d.doDDLJob(ctx, job) + err = d.doDDLJob(ctx, job) c.Assert(terror.ErrorEqual(err, infoschema.ErrDatabaseDropExists), IsTrue, Commentf("err %v", err)) // Drop a database without a table. - dbInfo1 := testSchemaInfo(c, d, "test1") + dbInfo1, err := testSchemaInfo(d, "test1") + c.Assert(err, IsNil) job = testCreateSchema(c, ctx, d, dbInfo1) testCheckSchemaState(c, d, dbInfo1, model.StatePublic) testCheckJobDone(c, d, job, true) @@ -201,12 +205,12 @@ func (s *testSchemaSuite) TestSchemaWaitJob(c *C) { c.Assert(err, IsNil) }() - d1 := testNewDDLAndStart( + d1, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d1.Stop() c.Assert(err, IsNil) @@ -214,12 +218,12 @@ func (s *testSchemaSuite) TestSchemaWaitJob(c *C) { testCheckOwner(c, d1, true) - d2 := testNewDDLAndStart( + d2, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease*4), ) + c.Assert(err, IsNil) defer func() { err := d2.Stop() c.Assert(err, IsNil) @@ -229,7 +233,8 @@ func (s *testSchemaSuite) TestSchemaWaitJob(c *C) { // d2 must not be owner. d2.ownerManager.RetireOwner() - dbInfo := testSchemaInfo(c, d2, "test_schema") + dbInfo, err := testSchemaInfo(d2, "test_schema") + c.Assert(err, IsNil) testCreateSchema(c, ctx, d2, dbInfo) testCheckSchemaState(c, d2, dbInfo, model.StatePublic) diff --git a/ddl/stat_test.go b/ddl/stat_test.go index 58cb113352805..13c4912ed693a 100644 --- a/ddl/stat_test.go +++ b/ddl/stat_test.go @@ -51,18 +51,19 @@ func (s *testSerialStatSuite) TestDDLStatsInfo(c *C) { c.Assert(err, IsNil) }() - d := testNewDDLAndStart( + d, err := testNewDDLAndStart( context.Background(), - c, WithStore(store), WithLease(testLease), ) + c.Assert(err, IsNil) defer func() { err := d.Stop() c.Assert(err, IsNil) }() - dbInfo := testSchemaInfo(c, d, "test_stat") + dbInfo, err := testSchemaInfo(d, "test_stat") + c.Assert(err, IsNil) testCreateSchema(c, testNewContext(d), d, dbInfo) tblInfo := testTableInfo(c, d, "t", 2) ctx := testNewContext(d) @@ -70,7 +71,7 @@ func (s *testSerialStatSuite) TestDDLStatsInfo(c *C) { t := testGetTable(c, d, dbInfo.ID, tblInfo.ID) // insert t values (1, 1), (2, 2), (3, 3) - _, err := t.AddRecord(ctx, types.MakeDatums(1, 1)) + _, err = t.AddRecord(ctx, types.MakeDatums(1, 1)) c.Assert(err, IsNil) _, err = t.AddRecord(ctx, types.MakeDatums(2, 2)) c.Assert(err, IsNil) diff --git a/ddl/table.go b/ddl/table.go index 4a4b2fb09d0f9..c2f217fc6eb21 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -1264,10 +1264,13 @@ func onAlterTablePartitionOptions(d *ddlCtx, t *meta.Meta, job *model.Job) (ver ptInfo := tblInfo.GetPartitionInfo() var partitionDef *model.PartitionDefinition definitions := ptInfo.Definitions + oldPartitionEnablesPlacement := false for i := range definitions { if partitionID == definitions[i].ID { - definitions[i].DirectPlacementOpts = placementSettings - definitions[i].PlacementPolicyRef = policyRefInfo + def := &definitions[i] + oldPartitionEnablesPlacement = def.PlacementPolicyRef != nil || def.DirectPlacementOpts != nil + def.DirectPlacementOpts = placementSettings + def.PlacementPolicyRef = policyRefInfo partitionDef = &definitions[i] break } @@ -1293,11 +1296,13 @@ func onAlterTablePartitionOptions(d *ddlCtx, t *meta.Meta, job *model.Job) (ver return ver, errors.Trace(err) } + if bundle == nil && oldPartitionEnablesPlacement { + bundle = placement.NewBundle(partitionDef.ID) + } + // Send the placement bundle to PD. if bundle != nil { err = infosync.PutRuleBundles(context.TODO(), []*placement.Bundle{bundle}) - } else { - err = dropRuleBundles(d, []int64{partitionDef.ID}) } if err != nil { @@ -1323,29 +1328,32 @@ func onAlterTablePlacement(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, return 0, err } + if _, err = checkPlacementPolicyRefValidAndCanNonValidJob(t, job, policyRefInfo); err != nil { + return 0, errors.Trace(err) + } + + oldTableEnablesPlacement := tblInfo.PlacementPolicyRef != nil || tblInfo.DirectPlacementOpts != nil tblInfo.PlacementPolicyRef = policyRefInfo tblInfo.DirectPlacementOpts = placementSettings - if _, err = checkPlacementPolicyRefValidAndCanNonValidJob(t, job, policyRefInfo); err != nil { - return 0, errors.Trace(err) + ver, err = updateVersionAndTableInfo(t, job, tblInfo, true) + if err != nil { + return ver, errors.Trace(err) } bundle, err := placement.NewTableBundle(t, tblInfo) if err != nil { job.State = model.JobStateCancelled - return 0, errors.Trace(err) + return ver, errors.Trace(err) } - ver, err = updateVersionAndTableInfo(t, job, tblInfo, true) - if err != nil { - return ver, errors.Trace(err) + if bundle == nil && oldTableEnablesPlacement { + bundle = placement.NewBundle(tblInfo.ID) } // Send the placement bundle to PD. if bundle != nil { err = infosync.PutRuleBundles(context.TODO(), []*placement.Bundle{bundle}) - } else { - err = dropRuleBundles(d, []int64{tblInfo.ID}) } if err != nil { diff --git a/ddl/table_split_test.go b/ddl/table_split_test.go index cc8ee498b2d13..a7f7462a8400e 100644 --- a/ddl/table_split_test.go +++ b/ddl/table_split_test.go @@ -17,35 +17,36 @@ package ddl_test import ( "context" "sync/atomic" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/tikv/client-go/v2/tikv" -) -type testDDLTableSplitSuite struct{} + "github.com/stretchr/testify/require" +) -var _ = Suite(&testDDLTableSplitSuite{}) +func TestTableSplit(t *testing.T) { + t.Parallel() -func (s *testDDLTableSplitSuite) TestTableSplit(c *C) { store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() session.SetSchemaLease(100 * time.Millisecond) session.DisableStats4Test() atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1) dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) + require.NoError(t, err) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // Synced split table region. tk.MustExec("set global tidb_scatter_region = 1") @@ -56,17 +57,17 @@ func (s *testDDLTableSplitSuite) TestTableSplit(c *C) { defer dom.Close() atomic.StoreUint32(&ddl.EnableSplitTableRegion, 0) infoSchema := dom.InfoSchema() - c.Assert(infoSchema, NotNil) - t, err := infoSchema.TableByName(model.NewCIStr("mysql"), model.NewCIStr("tidb")) - c.Assert(err, IsNil) - checkRegionStartWithTableID(c, t.Meta().ID, store.(kvStore)) + require.NotNil(t, infoSchema) + tbl, err := infoSchema.TableByName(model.NewCIStr("mysql"), model.NewCIStr("tidb")) + require.NoError(t, err) + checkRegionStartWithTableID(t, tbl.Meta().ID, store.(kvStore)) - t, err = infoSchema.TableByName(model.NewCIStr("test"), model.NewCIStr("t_part")) - c.Assert(err, IsNil) - pi := t.Meta().GetPartitionInfo() - c.Assert(pi, NotNil) + tbl, err = infoSchema.TableByName(model.NewCIStr("test"), model.NewCIStr("t_part")) + require.NoError(t, err) + pi := tbl.Meta().GetPartitionInfo() + require.NotNil(t, pi) for _, def := range pi.Definitions { - checkRegionStartWithTableID(c, def.ID, store.(kvStore)) + checkRegionStartWithTableID(t, def.ID, store.(kvStore)) } } @@ -74,14 +75,14 @@ type kvStore interface { GetRegionCache() *tikv.RegionCache } -func checkRegionStartWithTableID(c *C, id int64, store kvStore) { +func checkRegionStartWithTableID(t *testing.T, id int64, store kvStore) { regionStartKey := tablecodec.EncodeTablePrefix(id) var loc *tikv.KeyLocation var err error cache := store.GetRegionCache() loc, err = cache.LocateKey(tikv.NewBackoffer(context.Background(), 5000), regionStartKey) - c.Assert(err, IsNil) + require.NoError(t, err) // Region cache may be out of date, so we need to drop this expired region and load it again. cache.InvalidateCachedRegion(loc.Region) - c.Assert(loc.StartKey, BytesEquals, []byte(regionStartKey)) + require.Equal(t, []byte(regionStartKey), loc.StartKey) } diff --git a/ddl/table_test.go b/ddl/table_test.go index 7f577c8d1a916..b1919e7f9391a 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -349,14 +349,16 @@ func testGetTableWithError(d *ddl, schemaID, tableID int64) (table.Table, error) func (s *testTableSuite) SetUpSuite(c *C) { s.store = testCreateStore(c, "test_table") - s.d = testNewDDLAndStart( + ddl, err := testNewDDLAndStart( context.Background(), - c, WithStore(s.store), WithLease(testLease), ) + c.Assert(err, IsNil) + s.d = ddl - s.dbInfo = testSchemaInfo(c, s.d, "test_table") + s.dbInfo, err = testSchemaInfo(s.d, "test_table") + c.Assert(err, IsNil) testCreateSchema(c, testNewContext(s.d), s.d, s.dbInfo) } @@ -402,7 +404,8 @@ func (s *testTableSuite) TestTable(c *C) { testCheckJobDone(c, d, job, true) // for rename table - dbInfo1 := testSchemaInfo(c, s.d, "test_rename_table") + dbInfo1, err := testSchemaInfo(s.d, "test_rename_table") + c.Assert(err, IsNil) testCreateSchema(c, testNewContext(s.d), s.d, dbInfo1) job = testRenameTable(c, ctx, d, dbInfo1.ID, s.dbInfo.ID, s.dbInfo.Name, tblInfo) testCheckTableState(c, d, dbInfo1, tblInfo, model.StatePublic) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 44378b2a262a2..84c022a5b556b 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -18,6 +18,7 @@ import ( "fmt" "math" "sort" + "sync/atomic" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -235,7 +236,7 @@ func (builder *RequestBuilder) getKVPriority(sv *variable.SessionVars) int { } // SetFromSessionVars sets the following fields for "kv.Request" from session variables: -// "Concurrency", "IsolationLevel", "NotFillCache", "ReplicaRead", "SchemaVar". +// "Concurrency", "IsolationLevel", "NotFillCache", "TaskID", "Priority", "ReplicaRead", "ResourceGroupTagger". func (builder *RequestBuilder) SetFromSessionVars(sv *variable.SessionVars) *RequestBuilder { if builder.Request.Concurrency == 0 { // Concurrency may be set to 1 by SetDAGRequest @@ -246,7 +247,7 @@ func (builder *RequestBuilder) SetFromSessionVars(sv *variable.SessionVars) *Req builder.Request.TaskID = sv.StmtCtx.TaskID builder.Request.Priority = builder.getKVPriority(sv) builder.Request.ReplicaRead = sv.GetReplicaRead() - builder.SetResourceGroupTag(sv.StmtCtx) + builder.SetResourceGroupTagger(sv.StmtCtx) return builder } @@ -282,10 +283,10 @@ func (builder *RequestBuilder) SetFromInfoSchema(pis interface{}) *RequestBuilde return builder } -// SetResourceGroupTag sets the request resource group tag. -func (builder *RequestBuilder) SetResourceGroupTag(sc *stmtctx.StatementContext) *RequestBuilder { +// SetResourceGroupTagger sets the request resource group tagger. +func (builder *RequestBuilder) SetResourceGroupTagger(sc *stmtctx.StatementContext) *RequestBuilder { if variable.TopSQLEnabled() { - builder.Request.ResourceGroupTag = sc.GetResourceGroupTag() + builder.Request.ResourceGroupTagger = sc.GetResourceGroupTagger() } return builder } @@ -551,13 +552,25 @@ func PartitionHandlesToKVRanges(handles []kv.Handle) []kv.KeyRange { // IndexRangesToKVRanges converts index ranges to "KeyRange". func IndexRangesToKVRanges(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) ([]kv.KeyRange, error) { - return IndexRangesToKVRangesForTables(sc, []int64{tid}, idxID, ranges, fb) + return IndexRangesToKVRangesWithInterruptSignal(sc, tid, idxID, ranges, fb, nil, nil) +} + +// IndexRangesToKVRangesWithInterruptSignal converts index ranges to "KeyRange". +// The process can be interrupted by set `interruptSignal` to true. +func IndexRangesToKVRangesWithInterruptSignal(sc *stmtctx.StatementContext, tid, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback, memTracker *memory.Tracker, interruptSignal *atomic.Value) ([]kv.KeyRange, error) { + return indexRangesToKVRangesForTablesWithInterruptSignal(sc, []int64{tid}, idxID, ranges, fb, memTracker, interruptSignal) } // IndexRangesToKVRangesForTables converts indexes ranges to "KeyRange". func IndexRangesToKVRangesForTables(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback) ([]kv.KeyRange, error) { + return indexRangesToKVRangesForTablesWithInterruptSignal(sc, tids, idxID, ranges, fb, nil, nil) +} + +// IndexRangesToKVRangesForTablesWithInterruptSignal converts indexes ranges to "KeyRange". +// The process can be interrupted by set `interruptSignal` to true. +func indexRangesToKVRangesForTablesWithInterruptSignal(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range, fb *statistics.QueryFeedback, memTracker *memory.Tracker, interruptSignal *atomic.Value) ([]kv.KeyRange, error) { if fb == nil || fb.Hist == nil { - return indexRangesToKVWithoutSplit(sc, tids, idxID, ranges) + return indexRangesToKVWithoutSplit(sc, tids, idxID, ranges, memTracker, interruptSignal) } feedbackRanges := make([]*ranger.Range, 0, len(ranges)) for _, ran := range ranges { @@ -642,18 +655,37 @@ func VerifyTxnScope(txnScope string, physicalTableID int64, is infoschema.InfoSc return true } -func indexRangesToKVWithoutSplit(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range) ([]kv.KeyRange, error) { +func indexRangesToKVWithoutSplit(sc *stmtctx.StatementContext, tids []int64, idxID int64, ranges []*ranger.Range, memTracker *memory.Tracker, interruptSignal *atomic.Value) ([]kv.KeyRange, error) { krs := make([]kv.KeyRange, 0, len(ranges)) - for _, ran := range ranges { + const CheckSignalStep = 8 + var estimatedMemUsage int64 + // encodeIndexKey and EncodeIndexSeekKey is time-consuming, thus we need to + // check the interrupt signal periodically. + for i, ran := range ranges { low, high, err := encodeIndexKey(sc, ran) if err != nil { return nil, err } + if i == 0 { + estimatedMemUsage += int64(cap(low) + cap(high)) + } for _, tid := range tids { startKey := tablecodec.EncodeIndexSeekKey(tid, idxID, low) endKey := tablecodec.EncodeIndexSeekKey(tid, idxID, high) + if i == 0 { + estimatedMemUsage += int64(cap(startKey)) + int64(cap(endKey)) + } krs = append(krs, kv.KeyRange{StartKey: startKey, EndKey: endKey}) } + if i%CheckSignalStep == 0 { + if i == 0 && memTracker != nil { + estimatedMemUsage *= int64(len(ranges)) + memTracker.Consume(estimatedMemUsage) + } + if interruptSignal != nil && interruptSignal.Load().(bool) { + return nil, nil + } + } } return krs, nil } diff --git a/domain/domain.go b/domain/domain.go index 8ed2c18e58cd4..c5d8dc8246cd0 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -93,8 +93,8 @@ type Domain struct { serverID uint64 serverIDSession *concurrency.Session isLostConnectionToPD sync2.AtomicInt32 // !0: true, 0: false. - - onClose func() + renewLeaseCh chan func() // It is used to call the renewLease function of the cache table. + onClose func() } // loadInfoSchema loads infoschema at startTS. @@ -159,7 +159,7 @@ func (do *Domain) loadInfoSchema(startTS uint64) (infoschema.InfoSchema, bool, i return nil, false, currentSchemaVersion, nil, err } - newISBuilder, err := infoschema.NewBuilder(do.Store()).InitWithDBInfos(schemas, bundles, policies, neededSchemaVersion) + newISBuilder, err := infoschema.NewBuilder(do.Store(), do.renewLeaseCh).InitWithDBInfos(schemas, bundles, policies, neededSchemaVersion) if err != nil { return nil, false, currentSchemaVersion, nil, err } @@ -271,7 +271,7 @@ func (do *Domain) tryLoadSchemaDiffs(m *meta.Meta, usedVersion, newVersion int64 } diffs = append(diffs, diff) } - builder := infoschema.NewBuilder(do.Store()).InitWithOldInfoSchema(do.infoCache.GetLatest()) + builder := infoschema.NewBuilder(do.Store(), do.renewLeaseCh).InitWithOldInfoSchema(do.infoCache.GetLatest()) phyTblIDs := make([]int64, 0, len(diffs)) actions := make([]uint64, 0, len(diffs)) for _, diff := range diffs { @@ -287,6 +287,7 @@ func (do *Domain) tryLoadSchemaDiffs(m *meta.Meta, usedVersion, newVersion int64 actions = append(actions, uint64(1< ")) } -func (s *testSuiteAgg) TestGroupConcatAggr(c *C) { +func TestGroupConcatAggr(t *testing.T) { + t.Parallel() var err error // issue #5411 - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists test;") tk.MustExec("create table test(id int, name int)") @@ -570,21 +560,21 @@ func (s *testSuiteAgg) TestGroupConcatAggr(c *C) { tk.MustExec(fmt.Sprintf("set session group_concat_max_len=%v", maxLen)) result = tk.MustQuery("select group_concat(id ORDER BY name desc, id asc SEPARATOR '--') from test;") result.Check(testkit.Rows(expected[:maxLen])) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) } expected = "1--2--1--1--3--3" for maxLen := 4; maxLen < len(expected); maxLen++ { tk.MustExec(fmt.Sprintf("set session group_concat_max_len=%v", maxLen)) result = tk.MustQuery("select group_concat(id ORDER BY name asc, id desc SEPARATOR '--') from test;") result.Check(testkit.Rows(expected[:maxLen])) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) } expected = "500,200,30,20,10" for maxLen := 4; maxLen < len(expected); maxLen++ { tk.MustExec(fmt.Sprintf("set session group_concat_max_len=%v", maxLen)) result = tk.MustQuery("select group_concat(distinct name order by name desc) from test;") result.Check(testkit.Rows(expected[:maxLen])) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) } tk.MustExec(fmt.Sprintf("set session group_concat_max_len=%v", 1024)) @@ -606,8 +596,7 @@ func (s *testSuiteAgg) TestGroupConcatAggr(c *C) { tk.MustQuery("select 1, 2, 3, 4, 5 , group_concat(name, id ORDER BY 1 desc, id SEPARATOR '++') from test;").Check(testkit.Rows("1 2 3 4 5 5003++2003++301++201++202++101")) tk.MustQuery("select 1, 2, 3, 4, 5 , group_concat(name, id ORDER BY 2 desc, name SEPARATOR '++') from test;").Check(testkit.Rows("1 2 3 4 5 2003++5003++202++101++201++301")) err = tk.ExecToErr("select 1, 2, 3, 4, 5 , group_concat(name, id ORDER BY 3 desc, name SEPARATOR '++') from test;") - c.Assert(err.Error(), Equals, "[planner:1054]Unknown column '3' in 'order clause'") - + require.EqualError(t, err, "[planner:1054]Unknown column '3' in 'order clause'") // test Param Marker tk.MustExec(`prepare s1 from "select 1, 2, 3, 4, 5 , group_concat(name, id ORDER BY floor(id/?) desc, name SEPARATOR '++') from test";`) tk.MustExec("set @a=2;") @@ -618,7 +607,7 @@ func (s *testSuiteAgg) TestGroupConcatAggr(c *C) { tk.MustQuery("execute s1 using @a;").Check(testkit.Rows("1 2 3 4 5 2003++5003++202++101++201++301")) tk.MustExec("set @a=3;") err = tk.ExecToErr("execute s1 using @a;") - c.Assert(err.Error(), Equals, "[planner:1054]Unknown column '?' in 'order clause'") + require.EqualError(t, err, "[planner:1054]Unknown column '?' in 'order clause'") tk.MustExec("set @a=3.0;") tk.MustQuery("execute s1 using @a;").Check(testkit.Rows("1 2 3 4 5 101++202++201++301++2003++5003")) @@ -662,10 +651,13 @@ func (s *testSuiteAgg) TestGroupConcatAggr(c *C) { rows.Check(testkit.Rows("01234567", "12345")) } -func (s *testSuiteAgg) TestSelectDistinct(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSelectDistinct(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - s.fillData(tk, "select_distinct_test") + fillData(tk, "select_distinct_test") tk.MustExec("begin") r := tk.MustQuery("select distinct name from select_distinct_test;") @@ -674,8 +666,11 @@ func (s *testSuiteAgg) TestSelectDistinct(c *C) { } -func (s *testSuiteAgg) TestAggPushDown(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAggPushDown(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 t") tk.MustExec("create table t (a int, b int, c int)") @@ -702,8 +697,11 @@ func (s *testSuiteAgg) TestAggPushDown(c *C) { tk.MustQuery("select a, count(b) from (select * from t union all select * from tt) k group by a order by a").Check(testkit.Rows("1 2", "2 1")) } -func (s *testSuiteAgg) TestOnlyFullGroupBy(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestOnlyFullGroupBy(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set sql_mode = 'ONLY_FULL_GROUP_BY'") tk.MustExec("drop table if exists t") @@ -716,39 +714,39 @@ func (s *testSuiteAgg) TestOnlyFullGroupBy(c *C) { tk.MustQuery("select max(a), any_value(c) from t group by d;") // test incompatible with sql_mode = ONLY_FULL_GROUP_BY err := tk.ExecToErr("select * from t group by d") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) err = tk.ExecToErr("select b-c from t group by b+c") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) err = tk.ExecToErr("select (b-c)*(b+c), min(a) from t group by b+c, b-c") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) err = tk.ExecToErr("select b between c and d from t group by b,c") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) err = tk.ExecToErr("select case b when 1 then c when 2 then d else d end from t group by b,c") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) err = tk.ExecToErr("select c > (select b from t) from t group by b") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) err = tk.ExecToErr("select c is null from t group by b") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) err = tk.ExecToErr("select c is true from t group by b") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) err = tk.ExecToErr("select (c+b)*d from t group by c,d") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) err = tk.ExecToErr("select b in (c,d) from t group by b,c") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) err = tk.ExecToErr("select b like '%a' from t group by c") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) err = tk.ExecToErr("select c REGEXP '1.*' from t group by b") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) err = tk.ExecToErr("select -b from t group by c") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) err = tk.ExecToErr("select a, max(b) from t") - c.Assert(terror.ErrorEqual(err, plannercore.ErrMixOfGroupFuncAndFields), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrMixOfGroupFuncAndFields), "err %v", err) err = tk.ExecToErr("select sum(a)+b from t") - c.Assert(terror.ErrorEqual(err, plannercore.ErrMixOfGroupFuncAndFields), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrMixOfGroupFuncAndFields), "err %v", err) err = tk.ExecToErr("select count(b), c from t") - c.Assert(terror.ErrorEqual(err, plannercore.ErrMixOfGroupFuncAndFields), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrMixOfGroupFuncAndFields), "err %v", err) err = tk.ExecToErr("select distinct a, b, count(a) from t") - c.Assert(terror.ErrorEqual(err, plannercore.ErrMixOfGroupFuncAndFields), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrMixOfGroupFuncAndFields), "err %v", err) // test compatible with sql_mode = ONLY_FULL_GROUP_BY tk.MustQuery("select a from t group by a,b,c") tk.MustQuery("select b from t group by b") @@ -777,7 +775,7 @@ func (s *testSuiteAgg) TestOnlyFullGroupBy(c *C) { tk.MustQuery("select * from t group by b,d") // test functional depend on a unique null column err = tk.ExecToErr("select * from t group by b,c") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) // test functional dependency derived from keys in where condition tk.MustQuery("select * from t where c = d group by b, c") tk.MustQuery("select t.*, x.* from t, x where t.a = x.a group by t.a") @@ -785,7 +783,7 @@ func (s *testSuiteAgg) TestOnlyFullGroupBy(c *C) { tk.MustQuery("select t.*, x.* from t, x where t.b = x.a group by t.b, t.d") tk.MustQuery("select t.b, x.* from t, x where t.b = x.a group by t.b") err = tk.ExecToErr("select t.*, x.* from t, x where t.c = x.a group by t.b, t.c") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) // test functional dependency derived from keys in join tk.MustQuery("select t.*, x.* from t inner join x on t.a = x.a group by t.a") tk.MustQuery("select t.*, x.* from t inner join x on (t.b = x.b and t.d = x.d) group by t.b, x.d") @@ -795,9 +793,9 @@ func (s *testSuiteAgg) TestOnlyFullGroupBy(c *C) { tk.MustQuery("select x.b, t.* from t right join x on x.b = t.b group by x.b, t.d") tk.MustQuery("select x.b, t.* from t right join x on t.b = x.b group by x.b, t.d") err = tk.ExecToErr("select t.b, x.* from t right join x on t.b = x.b group by t.b, x.d") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) err = tk.ExecToErr("select t.b, x.* from t right join x on t.b = x.b group by t.b, x.d") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) // FixMe: test functional dependency of derived table // tk.MustQuery("select * from (select * from t) as e group by a") @@ -808,14 +806,17 @@ func (s *testSuiteAgg) TestOnlyFullGroupBy(c *C) { // test order by tk.MustQuery("select c from t group by c,d order by d") err = tk.ExecToErr("select c from t group by c order by d") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) // test ambiguous column err = tk.ExecToErr("select c from t,x group by t.c") - c.Assert(terror.ErrorEqual(err, plannercore.ErrAmbiguous), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrAmbiguous), "err %v", err) } -func (s *testSuiteAgg) TestIssue16279(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue16279(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set sql_mode = 'ONLY_FULL_GROUP_BY'") tk.MustExec("drop table if exists s") @@ -825,8 +826,11 @@ func (s *testSuiteAgg) TestIssue16279(c *C) { tk.MustQuery("select count(a) , date_format(a, '%Y-%m-%d') as xx from s group by xx") } -func (s *testSuiteAgg) TestIssue24676(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue24676(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set sql_mode = 'ONLY_FULL_GROUP_BY'") tk.MustExec("drop table if exists t1") @@ -837,11 +841,14 @@ func (s *testSuiteAgg) TestIssue24676(c *C) { tk.MustQuery("SELECT c1 FROM t1 GROUP BY c1 ORDER BY c1 ASC;") tk.MustQuery("SELECT ((floor(((`c1` - 0.0) / 50000)) * 50000) + 0.0) AS `c1` FROM `t1` GROUP BY ((floor(((`c1` - 0.0) / 50000)) * 50000) + 0.0) ORDER BY ((floor(((`c1` - 0.0) / 50000)) * 50000) + 0.0) ASC;") err := tk.ExecToErr("SELECT ((floor(((`c1` - 10) / 300)) * 50000) + 0.0) AS `c1` FROM `t1` GROUP BY ((floor(((`c1` - 0.0) / 50000)) * 50000) + 0.0) ORDER BY ((floor(((`c1` - 0.0) / 50000)) * 50000) + 0.0) ASC;") - c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), "err %v", err) } -func (s *testSuiteAgg) TestAggPushDownPartitionTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAggPushDownPartitionTable(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 t1") tk.MustExec(`CREATE TABLE t1 ( @@ -872,8 +879,11 @@ func (s *testSuiteAgg) TestAggPushDownPartitionTable(c *C) { "99 102")) } -func (s *testSuiteAgg) TestIssue13652(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue13652(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set sql_mode = 'ONLY_FULL_GROUP_BY'") tk.MustExec("drop table if exists t") @@ -883,11 +893,14 @@ func (s *testSuiteAgg) TestIssue13652(c *C) { tk.MustQuery("select a from t group by +a") tk.MustQuery("select a from t group by ((+a))") _, err := tk.Exec("select a from t group by (-a)") - c.Assert(err.Error(), Equals, "[planner:1055]Expression #1 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.t.a' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by") + require.EqualError(t, err, "[planner:1055]Expression #1 of SELECT list is not in GROUP BY clause and contains nonaggregated column 'test.t.a' which is not functionally dependent on columns in GROUP BY clause; this is incompatible with sql_mode=only_full_group_by") } -func (s *testSuiteAgg) TestIssue14947(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue14947(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set sql_mode = 'ONLY_FULL_GROUP_BY'") tk.MustExec("drop table if exists t") @@ -895,9 +908,12 @@ func (s *testSuiteAgg) TestIssue14947(c *C) { tk.MustQuery("select ((+a+1)) as tmp from t group by tmp") } -func (s *testSuiteAgg) TestHaving(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - +func TestHaving(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("set sql_mode = 'STRICT_TRANS_TABLES,NO_ENGINE_SUBSTITUTION'") tk.MustExec("drop table if exists t") tk.MustExec("create table t (c1 int, c2 int, c3 int)") @@ -920,8 +936,12 @@ func (s *testSuiteAgg) TestHaving(c *C) { tk.MustQuery("select 1 from t group by c1 having sum(abs(c2 + c3)) = c1").Check(testkit.Rows("1")) } -func (s *testSuiteAgg) TestIssue26496(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue26496(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 UK_NSPRE_19416") tk.MustExec("CREATE TABLE `UK_NSPRE_19416` ( `COL1` binary(20) DEFAULT NULL, `COL2` varchar(20) DEFAULT NULL, `COL4` datetime DEFAULT NULL, `COL3` bigint(20) DEFAULT NULL, `COL5` float DEFAULT NULL, UNIQUE KEY `UK_COL1` (`COL1`)) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin") @@ -929,8 +949,12 @@ func (s *testSuiteAgg) TestIssue26496(c *C) { tk.MustQuery("select t1.col1, count(t2.col1) from UK_NSPRE_19416 as t1 left join UK_NSPRE_19416 as t2 on t1.col1 = t2.col1 where t1.col1 in (0x550C4A4390BA14FD6D382DD29063E10210C99381, 0x004DAD637B37CC4A9742484AB93F97EDE2AB8BD5, 0xC5B428E2EBC1B78F0B183899A8DF55C88A333F86) group by t1.col1, t2.col1 having t1.col1 in (0x9B4B48FEBA9225BACF8F9ADEAEE810AEC26DC7A2, 0x25A6C4FAD832F8E0267AAA504CFAE767565C8B84, 0xE26E5B0080EC5A8156DACE67D13B239500E540E6)").Check(nil) } -func (s *testSuiteAgg) TestAggEliminator(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestAggEliminator(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 int primary key, b int)") tk.MustQuery("select min(a), min(a) from t").Check(testkit.Rows(" ")) @@ -942,10 +966,14 @@ func (s *testSuiteAgg) TestAggEliminator(c *C) { tk.MustQuery("select group_concat(b, b) from t group by a").Sort().Check(testkit.Rows("-1-1", "-2-2", "11", "")) } -func (s *testSuiteAgg) TestClusterIndexMaxMinEliminator(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestClusterIndexMaxMinEliminator(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 t;") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("create table t (a int, b int, c int, primary key(a, b));") for i := 0; i < 10+1; i++ { tk.MustExec("insert into t values (?, ?, ?)", i, i, i) @@ -955,8 +983,12 @@ func (s *testSuiteAgg) TestClusterIndexMaxMinEliminator(c *C) { tk.MustQuery("select min(a), max(a), min(b), max(b) from t;").Check(testkit.Rows("0 10 0 10")) } -func (s *testSuiteAgg) TestMaxMinFloatScalaFunc(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestMaxMinFloatScalaFunc(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 T;`) tk.MustExec(`CREATE TABLE T(A VARCHAR(10), B VARCHAR(10), C FLOAT);`) @@ -965,8 +997,12 @@ func (s *testSuiteAgg) TestMaxMinFloatScalaFunc(c *C) { tk.MustQuery(`SELECT MIN(CASE B WHEN 'val_b' THEN C ELSE 0 END) val_b FROM T WHERE cast(A as signed) = 0 GROUP BY a;`).Check(testkit.Rows("12.190999984741211")) } -func (s *testSuiteAgg) TestBuildProjBelowAgg(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestBuildProjBelowAgg(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 t;") tk.MustExec("create table t (i int);") tk.MustExec("insert into t values (1), (1), (1),(2),(3),(2),(3),(2),(3);") @@ -977,8 +1013,12 @@ func (s *testSuiteAgg) TestBuildProjBelowAgg(c *C) { "4 3 18 7,7,7 8")) } -func (s *testSuiteAgg) TestInjectProjBelowTopN(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestInjectProjBelowTopN(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 t;") tk.MustExec("create table t (i int);") tk.MustExec("insert into t values (1), (1), (1),(2),(3),(2),(3),(2),(3);") @@ -986,17 +1026,20 @@ func (s *testSuiteAgg) TestInjectProjBelowTopN(c *C) { input []string output [][]string ) - s.testData.GetTestCases(c, &input, &output) + aggMergeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { - output[i] = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i]...)) } } -func (s *testSuiteAgg) TestFirstRowEnum(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestFirstRowEnum(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 t;`) tk.MustExec(`create table t(a enum('a', 'b'));`) @@ -1006,8 +1049,12 @@ func (s *testSuiteAgg) TestFirstRowEnum(c *C) { )) } -func (s *testSuiteAgg) TestAggJSON(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestAggJSON(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 t;`) tk.MustExec(`create table t(a datetime, b json, index idx(a));`) tk.MustExec(`insert into t values('2019-03-20 21:50:00', '["a", "b", 1]');`) @@ -1055,8 +1102,12 @@ func (s *testSuiteAgg) TestAggJSON(c *C) { )) } -func (s *testSuiteAgg) TestIssue10099(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue10099(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 t") tk.MustExec("create table t(a char(10), b char(10))") tk.MustExec("insert into t values('1', '222'), ('12', '22')") @@ -1064,16 +1115,24 @@ func (s *testSuiteAgg) TestIssue10099(c *C) { tk.MustQuery("select approx_count_distinct( a, b) from t").Check(testkit.Rows("2")) } -func (s *testSuiteAgg) TestIssue10098(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue10098(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 t;`) tk.MustExec("create table t(a char(10), b char(10))") tk.MustExec("insert into t values('1', '222'), ('12', '22')") tk.MustQuery("select group_concat(distinct a, b) from t").Check(testkit.Rows("1222,1222")) } -func (s *testSuiteAgg) TestIssue10608(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue10608(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 t, s;`) tk.MustExec("create table t(a int)") tk.MustExec("create table s(a int, b int)") @@ -1084,9 +1143,15 @@ func (s *testSuiteAgg) TestIssue10608(c *C) { } -func (s *testSuiteAgg) TestIssue12759HashAggCalledByApply(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.Se.GetSessionVars().SetHashAggFinalConcurrency(4) +func TestIssue12759HashAggCalledByApply(t *testing.T) { + t.Parallel() + + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.Session().GetSessionVars().SetHashAggFinalConcurrency(4) tk.MustExec(`insert into mysql.opt_rule_blacklist value("decorrelate");`) defer func() { tk.MustExec(`delete from mysql.opt_rule_blacklist where name = "decorrelate";`) @@ -1101,17 +1166,21 @@ func (s *testSuiteAgg) TestIssue12759HashAggCalledByApply(c *C) { input []string output [][]string ) - s.testData.GetTestCases(c, &input, &output) + aggMergeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { - output[i] = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i]...)) } } -func (s *testSuiteAgg) TestPR15242ShallowCopy(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestPR15242ShallowCopy(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 t;`) tk.MustExec(`create table t(a json);`) tk.MustExec(`insert into t values ('{"id": 1,"score":23}');`) @@ -1119,21 +1188,25 @@ func (s *testSuiteAgg) TestPR15242ShallowCopy(c *C) { tk.MustExec(`insert into t values ('{"id": 1,"score":233}');`) tk.MustExec(`insert into t values ('{"id": 2,"score":233}');`) tk.MustExec(`insert into t values ('{"id": 3,"score":233}');`) - tk.Se.GetSessionVars().MaxChunkSize = 2 + tk.Session().GetSessionVars().MaxChunkSize = 2 tk.MustQuery(`select max(JSON_EXTRACT(a, '$.score')) as max_score,JSON_EXTRACT(a,'$.id') as id from t group by id order by id;`).Check(testkit.Rows("233 1", "233 2", "233 3")) } -func (s *testSuiteAgg) TestIssue15690(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.Se.GetSessionVars().MaxChunkSize = 2 +func TestIssue15690(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.Session().GetSessionVars().MaxChunkSize = 2 // check for INT type tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(a int);`) tk.MustExec(`insert into t values(null),(null);`) tk.MustExec(`insert into t values(0),(2),(2),(4),(8);`) tk.MustQuery(`select /*+ stream_agg() */ distinct * from t;`).Check(testkit.Rows("", "0", "2", "4", "8")) - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0)) + require.Equal(t, tk.Session().GetSessionVars().StmtCtx.WarningCount(), uint16(0)) // check for FLOAT type tk.MustExec(`drop table if exists t;`) @@ -1141,15 +1214,14 @@ func (s *testSuiteAgg) TestIssue15690(c *C) { tk.MustExec(`insert into t values(null),(null),(null),(null);`) tk.MustExec(`insert into t values(1.1),(1.1);`) tk.MustQuery(`select /*+ stream_agg() */ distinct * from t;`).Check(testkit.Rows("", "1.1")) - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0)) - + require.Equal(t, tk.Session().GetSessionVars().StmtCtx.WarningCount(), uint16(0)) // check for DECIMAL type tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(a decimal(5,1));`) tk.MustExec(`insert into t values(null),(null),(null);`) tk.MustExec(`insert into t values(1.1),(2.2),(2.2);`) tk.MustQuery(`select /*+ stream_agg() */ distinct * from t;`).Check(testkit.Rows("", "1.1", "2.2")) - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0)) + require.Equal(t, tk.Session().GetSessionVars().StmtCtx.WarningCount(), uint16(0)) // check for DATETIME type tk.MustExec(`drop table if exists t;`) @@ -1157,14 +1229,14 @@ func (s *testSuiteAgg) TestIssue15690(c *C) { tk.MustExec(`insert into t values(null);`) tk.MustExec(`insert into t values("2019-03-20 21:50:00"),("2019-03-20 21:50:01"), ("2019-03-20 21:50:00");`) tk.MustQuery(`select /*+ stream_agg() */ distinct * from t;`).Check(testkit.Rows("", "2019-03-20 21:50:00", "2019-03-20 21:50:01")) - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0)) + require.Equal(t, tk.Session().GetSessionVars().StmtCtx.WarningCount(), uint16(0)) // check for JSON type tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(a json);`) tk.MustExec(`insert into t values(null),(null),(null),(null);`) tk.MustQuery(`select /*+ stream_agg() */ distinct * from t;`).Check(testkit.Rows("")) - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0)) + require.Equal(t, tk.Session().GetSessionVars().StmtCtx.WarningCount(), uint16(0)) // check for char type tk.MustExec(`drop table if exists t;`) @@ -1172,12 +1244,16 @@ func (s *testSuiteAgg) TestIssue15690(c *C) { tk.MustExec(`insert into t values(null),(null),(null),(null);`) tk.MustExec(`insert into t values('a'),('b');`) tk.MustQuery(`select /*+ stream_agg() */ distinct * from t;`).Check(testkit.Rows("", "a", "b")) - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0)) + require.Equal(t, tk.Session().GetSessionVars().StmtCtx.WarningCount(), uint16(0)) } -func (s *testSuiteAgg) TestIssue15958(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.Se.GetSessionVars().MaxChunkSize = 2 +func TestIssue15958(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.Session().GetSessionVars().MaxChunkSize = 2 tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(y year);`) tk.MustExec(`insert into t values (2020), (2000), (2050);`) @@ -1185,8 +1261,11 @@ func (s *testSuiteAgg) TestIssue15958(c *C) { tk.MustQuery(`select avg(y) from t`).Check(testkit.Rows("2023.3333")) } -func (s *testSuiteAgg) TestIssue17216(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue17216(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 t1") tk.MustExec(`CREATE TABLE t1 ( @@ -1197,7 +1276,8 @@ func (s *testSuiteAgg) TestIssue17216(c *C) { tk.MustQuery("SELECT count(distinct col1) FROM t1").Check(testkit.Rows("48")) } -func (s *testSuiteAgg) TestHashAggRuntimeStat(c *C) { +func TestHashAggRuntimeStat(t *testing.T) { + t.Parallel() partialInfo := &executor.AggWorkerInfo{ Concurrency: 5, WallTime: int64(time.Second * 20), @@ -1229,12 +1309,11 @@ func (s *testSuiteAgg) TestHashAggRuntimeStat(c *C) { }) } expect := "partial_worker:{wall_time:20s, concurrency:5, task_num:25, tot_wait:10s, tot_exec:5s, tot_time:10s, max:4s, p95:4s}, final_worker:{wall_time:10s, concurrency:8, task_num:40, tot_wait:16ms, tot_exec:8ms, tot_time:28ms, max:7ms, p95:7ms}" - c.Assert(stats.String(), Equals, expect) - c.Assert(stats.String(), Equals, expect) - c.Assert(stats.Clone().String(), Equals, expect) + require.Equal(t, expect, stats.String()) + require.Equal(t, expect, stats.Clone().String()) stats.Merge(stats.Clone()) expect = "partial_worker:{wall_time:40s, concurrency:5, task_num:50, tot_wait:20s, tot_exec:10s, tot_time:20s, max:4s, p95:4s}, final_worker:{wall_time:20s, concurrency:8, task_num:80, tot_wait:32ms, tot_exec:16ms, tot_time:56ms, max:7ms, p95:7ms}" - c.Assert(stats.String(), Equals, expect) + require.Equal(t, expect, stats.String()) } func reconstructParallelGroupConcatResult(rows [][]interface{}) []string { @@ -1256,8 +1335,11 @@ func reconstructParallelGroupConcatResult(rows [][]interface{}) []string { return data } -func (s *testSuiteAgg) TestParallelStreamAggGroupConcat(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestParallelStreamAggGroupConcat(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 t;") tk.MustExec("CREATE TABLE t(a bigint, b bigint);") @@ -1291,19 +1373,22 @@ func (s *testSuiteAgg) TestParallelStreamAggGroupConcat(c *C) { break } } - c.Assert(ok, Equals, true) + require.True(t, ok) obtained := reconstructParallelGroupConcatResult(tk.MustQuery(sql).Rows()) - c.Assert(len(obtained), Equals, len(expected)) + require.Equal(t, len(expected), len(obtained)) for i := 0; i < len(obtained); i++ { - c.Assert(obtained[i], Equals, expected[i]) + require.Equal(t, expected[i], obtained[i]) } } } } -func (s *testSuiteAgg) TestIssue20658(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("use test;") +func TestIssue20658(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") aggFuncs := []string{"count(a)", "sum(a)", "avg(a)", "max(a)", "min(a)", "bit_or(a)", "bit_xor(a)", "bit_and(a)", "var_pop(a)", "var_samp(a)", "stddev_pop(a)", "stddev_samp(a)", "approx_count_distinct(a)", "approx_percentile(a, 7)"} sqlFormat := "select /*+ stream_agg() */ %s from t group by b;" @@ -1342,7 +1427,7 @@ func (s *testSuiteAgg) TestIssue20658(c *C) { continue } v, err := strconv.ParseFloat(rowStr, 64) - c.Assert(err, IsNil, cmt) + require.NoError(t, err, cmt) ret[i] = v } sort.Float64s(ret) @@ -1364,74 +1449,22 @@ func (s *testSuiteAgg) TestIssue20658(c *C) { break } } - c.Assert(ok, Equals, true, comment) + require.True(t, ok, comment) rows := mustParseAndSort(tk.MustQuery(sql).Rows(), comment) - c.Assert(len(rows), Equals, len(expected), comment) + require.Equal(t, len(expected), len(rows), comment) for i := range rows { - c.Assert(math.Abs(rows[i]-expected[i]), Less, 1e-3, comment) + require.Less(t, math.Abs(rows[i]-expected[i]), 1e-3, comment) } } } } -func (s *testSerialSuite) TestRandomPanicAggConsume(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("set @@tidb_max_chunk_size=32") - tk.MustExec("set @@tidb_init_chunk_size=1") - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int)") - for i := 0; i <= 1000; i++ { - tk.MustExec(fmt.Sprintf("insert into t values(%v),(%v),(%v)", i, i, i)) - } - - fpName := "github.com/pingcap/tidb/executor/ConsumeRandomPanic" - c.Assert(failpoint.Enable(fpName, "5%panic(\"ERROR 1105 (HY000): Out Of Memory Quota![conn_id=1]\")"), IsNil) - defer func() { - c.Assert(failpoint.Disable(fpName), IsNil) - }() - - // Test 10 times panic for each AggExec. - var res sqlexec.RecordSet - for i := 1; i <= 10; i++ { - var err error - for err == nil { - // Test paralleled hash agg. - res, err = tk.Exec("select /*+ HASH_AGG() */ count(a) from t group by a") - if err == nil { - _, err = session.GetRows4Test(context.Background(), tk.Se, res) - c.Assert(res.Close(), IsNil) - } - } - c.Assert(err.Error(), Equals, "failpoint panic: ERROR 1105 (HY000): Out Of Memory Quota![conn_id=1]") - - err = nil - for err == nil { - // Test unparalleled hash agg. - res, err = tk.Exec("select /*+ HASH_AGG() */ count(distinct a) from t") - if err == nil { - _, err = session.GetRows4Test(context.Background(), tk.Se, res) - c.Assert(res.Close(), IsNil) - } - } - c.Assert(err.Error(), Equals, "failpoint panic: ERROR 1105 (HY000): Out Of Memory Quota![conn_id=1]") - - err = nil - for err == nil { - // Test stream agg. - res, err = tk.Exec("select /*+ STREAM_AGG() */ count(a) from t") - if err == nil { - _, err = session.GetRows4Test(context.Background(), tk.Se, res) - c.Assert(res.Close(), IsNil) - } - } - c.Assert(err.Error(), Equals, "failpoint panic: ERROR 1105 (HY000): Out Of Memory Quota![conn_id=1]") - } -} - -func (s *testSuiteAgg) TestIssue23277(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue23277(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 t;") @@ -1462,8 +1495,11 @@ func (s *testSuiteAgg) TestIssue23277(c *C) { } // https://github.com/pingcap/tidb/issues/23314 -func (s *testSuiteAgg) TestIssue23314(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue23314(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 t1") tk.MustExec("create table t1(col1 time(2) NOT NULL)") @@ -1471,42 +1507,3 @@ func (s *testSuiteAgg) TestIssue23314(c *C) { res := tk.MustQuery("select col1 from t1 group by col1") res.Check(testkit.Rows("16:40:20.01")) } - -func (s *testSerialSuite) TestAggInDisk(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("set tidb_hashagg_final_concurrency = 1;") - tk.MustExec("set tidb_hashagg_partial_concurrency = 1;") - tk.MustExec("set tidb_mem_quota_query = 4194304") - tk.MustExec("drop table if exists t1") - tk.MustExec("create table t(a int)") - sql := "insert into t values (0)" - for i := 1; i <= 200; i++ { - sql += fmt.Sprintf(",(%v)", i) - } - sql += ";" - tk.MustExec(sql) - rows := tk.MustQuery("desc analyze select /*+ HASH_AGG() */ avg(t1.a) from t t1 join t t2 group by t1.a, t2.a;").Rows() - for _, row := range rows { - length := len(row) - line := fmt.Sprintf("%v", row) - disk := fmt.Sprintf("%v", row[length-1]) - if strings.Contains(line, "HashAgg") { - c.Assert(strings.Contains(disk, "0 Bytes"), IsFalse) - c.Assert(strings.Contains(disk, "MB") || - strings.Contains(disk, "KB") || - strings.Contains(disk, "Bytes"), IsTrue) - } - } - - // Add code cover - // Test spill chunk. Add a line to avoid tmp spill chunk is always full. - tk.MustExec("insert into t values(0)") - tk.MustQuery("select sum(tt.b) from ( select /*+ HASH_AGG() */ avg(t1.a) as b from t t1 join t t2 group by t1.a, t2.a) as tt").Check( - testkit.Rows("4040100.0000")) - // Test no groupby and no data. - tk.MustExec("drop table t;") - tk.MustExec("create table t(c int, c1 int);") - tk.MustQuery("select /*+ HASH_AGG() */ count(c) from t;").Check(testkit.Rows("0")) - tk.MustQuery("select /*+ HASH_AGG() */ count(c) from t group by c1;").Check(testkit.Rows()) -} diff --git a/executor/analyze.go b/executor/analyze.go index 0763a49209d35..9414b2fbade1e 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -389,7 +389,7 @@ func (e *AnalyzeIndexExec) fetchAnalyzeResult(ranges []*ranger.Range, isNullRang } else { kvReqBuilder = builder.SetIndexRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.tableID.GetStatisticsID()}, e.idxInfo.ID, ranges) } - kvReqBuilder.SetResourceGroupTag(e.ctx.GetSessionVars().StmtCtx) + kvReqBuilder.SetResourceGroupTagger(e.ctx.GetSessionVars().StmtCtx) kvReq, err := kvReqBuilder. SetAnalyzeRequest(e.analyzePB). SetStartTS(e.snapshot). @@ -750,7 +750,7 @@ func (e *AnalyzeColumnsExec) open(ranges []*ranger.Range) error { func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectResult, error) { var builder distsql.RequestBuilder reqBuilder := builder.SetHandleRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.TableID.GetStatisticsID()}, e.handleCols != nil && !e.handleCols.IsInt(), ranges, nil) - builder.SetResourceGroupTag(e.ctx.GetSessionVars().StmtCtx) + builder.SetResourceGroupTagger(e.ctx.GetSessionVars().StmtCtx) // Always set KeepOrder of the request to be true, in order to compute // correct `correlation` of columns. kvReq, err := reqBuilder. @@ -1853,7 +1853,7 @@ func (e *AnalyzeFastExec) handleScanTasks(bo *tikv.Backoffer) (keysSize int, err if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } - setResourceGroupTagForTxn(e.ctx.GetSessionVars().StmtCtx, snapshot) + setResourceGroupTaggerForTxn(e.ctx.GetSessionVars().StmtCtx, snapshot) for _, t := range e.scanTasks { iter, err := snapshot.Iter(kv.Key(t.StartKey), kv.Key(t.EndKey)) if err != nil { @@ -1874,7 +1874,7 @@ func (e *AnalyzeFastExec) handleSampTasks(workID int, step uint32, err *error) { snapshot.SetOption(kv.NotFillCache, true) snapshot.SetOption(kv.IsolationLevel, kv.SI) snapshot.SetOption(kv.Priority, kv.PriorityLow) - setResourceGroupTagForTxn(e.ctx.GetSessionVars().StmtCtx, snapshot) + setResourceGroupTaggerForTxn(e.ctx.GetSessionVars().StmtCtx, snapshot) readReplicaType := e.ctx.GetSessionVars().GetReplicaRead() if readReplicaType.IsFollowerRead() { snapshot.SetOption(kv.ReplicaRead, readReplicaType) @@ -2048,7 +2048,7 @@ func (e *AnalyzeFastExec) runTasks() ([]*statistics.Histogram, []*statistics.CMS func (e *AnalyzeFastExec) buildStats() (hists []*statistics.Histogram, cms []*statistics.CMSketch, topNs []*statistics.TopN, fms []*statistics.FMSketch, err error) { // To set rand seed, it's for unit test. // To ensure that random sequences are different in non-test environments, RandSeed must be set time.Now(). - if RandSeed == 1 { + if atomic.LoadInt64(&RandSeed) == 1 { atomic.StoreInt64(&e.randSeed, time.Now().UnixNano()) } else { atomic.StoreInt64(&e.randSeed, RandSeed) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 32a9c00edddc4..f8c574f36d53a 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -20,6 +20,7 @@ import ( "strconv" "strings" "sync" + "sync/atomic" "time" . "github.com/pingcap/check" @@ -309,7 +310,7 @@ func (s *testFastAnalyze) TestAnalyzeFastSample(c *C) { c.Assert(err, IsNil) defer dom.Close() tk := testkit.NewTestKit(c, store) - executor.RandSeed = 123 + atomic.StoreInt64(&executor.RandSeed, 123) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -370,7 +371,7 @@ func (s *testFastAnalyze) TestAnalyzeFastSample(c *C) { samples := mockExec.Collectors[i].Samples c.Assert(len(samples), Equals, 20) for j := 1; j < 20; j++ { - cmp, err := samples[j].Value.CompareDatum(tk.Se.GetSessionVars().StmtCtx, &samples[j-1].Value) + cmp, err := samples[j].Value.Compare(tk.Se.GetSessionVars().StmtCtx, &samples[j-1].Value, collate.GetBinaryCollator()) c.Assert(err, IsNil) c.Assert(cmp, Greater, 0) } @@ -380,7 +381,7 @@ func (s *testFastAnalyze) TestAnalyzeFastSample(c *C) { func checkHistogram(sc *stmtctx.StatementContext, hg *statistics.Histogram) (bool, error) { for i := 0; i < len(hg.Buckets); i++ { lower, upper := hg.GetLower(i), hg.GetUpper(i) - cmp, err := upper.CompareDatum(sc, lower) + cmp, err := upper.Compare(sc, lower, collate.GetBinaryCollator()) if cmp < 0 || err != nil { return false, err } @@ -388,7 +389,7 @@ func checkHistogram(sc *stmtctx.StatementContext, hg *statistics.Histogram) (boo continue } previousUpper := hg.GetUpper(i - 1) - cmp, err = lower.CompareDatum(sc, previousUpper) + cmp, err = lower.Compare(sc, previousUpper, collate.GetBinaryCollator()) if cmp <= 0 || err != nil { return false, err } @@ -418,7 +419,7 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) { dom.SetStatsUpdating(true) defer dom.Close() tk := testkit.NewTestKit(c, store) - executor.RandSeed = 123 + atomic.StoreInt64(&executor.RandSeed, 123) tk.MustExec("use test") tk.MustExec("drop table if exists t") diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 93c037b6775b6..c30bf507d6d9d 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -149,7 +149,7 @@ func (e *BatchPointGetExec) Open(context.Context) error { }, }) } - setResourceGroupTagForTxn(stmtCtx, snapshot) + setResourceGroupTaggerForTxn(stmtCtx, snapshot) var batchGetter kv.BatchGetter = snapshot if txn.Valid() { lock := e.tblInfo.Lock diff --git a/executor/brie_test.go b/executor/brie_test.go index 12b61b0e51146..0833a4ca85f39 100644 --- a/executor/brie_test.go +++ b/executor/brie_test.go @@ -18,32 +18,29 @@ import ( "context" "fmt" "strings" - gotime "time" + "testing" + "time" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/mock" - - . "github.com/pingcap/check" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" ) -type testBRIESuite struct{} - -var _ = Suite(&testBRIESuite{}) - -func (s *testBRIESuite) TestGlueGetVersion(c *C) { +func TestGlueGetVersion(t *testing.T) { + t.Parallel() g := tidbGlueSession{} version := g.GetVersion() - c.Assert(version, Matches, `(.|\n)*Release Version(.|\n)*`) - c.Assert(version, Matches, `(.|\n)*Git Commit Hash(.|\n)*`) - c.Assert(version, Matches, `(.|\n)*GoVersion(.|\n)*`) + require.Contains(t, version, `Release Version`) + require.Contains(t, version, `Git Commit Hash`) + require.Contains(t, version, `GoVersion`) } func brieTaskInfoToResult(info *brieTaskInfo) string { @@ -63,13 +60,14 @@ func brieTaskInfoToResult(info *brieTaskInfo) string { return strings.Join(arr, ", ") + "\n" } -func fetchShowBRIEResult(c *C, e *ShowExec, brieColTypes []*types.FieldType) string { +func fetchShowBRIEResult(t *testing.T, e *ShowExec, brieColTypes []*types.FieldType) string { e.result = newFirstChunk(e) - c.Assert(e.fetchShowBRIE(ast.BRIEKindBackup), IsNil) + require.NoError(t, e.fetchShowBRIE(ast.BRIEKindBackup)) return e.result.ToString(brieColTypes) } -func (s *testBRIESuite) TestFetchShowBRIE(c *C) { +func TestFetchShowBRIE(t *testing.T) { + t.Parallel() // Compose a mocked session manager. ps := make([]*util.ProcessInfo, 0, 1) pi := &util.ProcessInfo{ @@ -95,9 +93,9 @@ func (s *testBRIESuite) TestFetchShowBRIE(c *C) { p := parser.New() p.SetParserConfig(parser.ParserConfig{EnableWindowFunction: true, EnableStrictDoubleTypeCheck: true}) stmt, err := p.ParseOneStmt("show backups", "", "") - c.Assert(err, IsNil) + require.NoError(t, err) plan, _, err := core.BuildLogicalPlanForTest(ctx, sctx, stmt, infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable(), core.MockView()})) - c.Assert(err, IsNil) + require.NoError(t, err) schema := plan.Schema() // Compose executor. @@ -105,10 +103,10 @@ func (s *testBRIESuite) TestFetchShowBRIE(c *C) { baseExecutor: newBaseExecutor(sctx, schema, 0), Tp: ast.ShowBackups, } - c.Assert(e.Open(ctx), IsNil) + require.NoError(t, e.Open(ctx)) tp := mysql.TypeDatetime - lateTime := types.NewTime(types.FromGoTime(gotime.Now().Add(-outdatedDuration.Duration+1)), tp, 0) + lateTime := types.NewTime(types.FromGoTime(time.Now().Add(-outdatedDuration.Duration+1)), tp, 0) brieColTypes := make([]*types.FieldType, 0, len(schema.Columns)) for _, col := range schema.Columns { brieColTypes = append(brieColTypes, col.RetType) @@ -127,20 +125,20 @@ func (s *testBRIESuite) TestFetchShowBRIE(c *C) { info1Res := brieTaskInfoToResult(info1) globalBRIEQueue.registerTask(ctx, info1) - c.Assert(fetchShowBRIEResult(c, e, brieColTypes), Equals, info1Res) + require.Equal(t, info1Res, fetchShowBRIEResult(t, e, brieColTypes)) // Query again, this info should already have been cleaned - c.Assert(fetchShowBRIEResult(c, e, brieColTypes), HasLen, 0) + require.Len(t, fetchShowBRIEResult(t, e, brieColTypes), 0) // Register this task again, we should be able to fetch this info globalBRIEQueue.registerTask(ctx, info1) - c.Assert(fetchShowBRIEResult(c, e, brieColTypes), Equals, info1Res) + require.Equal(t, info1Res, fetchShowBRIEResult(t, e, brieColTypes)) // Query again, we should be able to fetch this info again, because we have cleared in last clearInterval - c.Assert(fetchShowBRIEResult(c, e, brieColTypes), Equals, info1Res) + require.Equal(t, info1Res, fetchShowBRIEResult(t, e, brieColTypes)) // Reset clear time, we should only fetch info2 this time. - globalBRIEQueue.lastClearTime = gotime.Now().Add(-clearInterval - gotime.Second) + globalBRIEQueue.lastClearTime = time.Now().Add(-clearInterval - time.Second) currTime := types.CurrentTime(tp) info2 := &brieTaskInfo{ kind: ast.BRIEKindBackup, @@ -154,5 +152,5 @@ func (s *testBRIESuite) TestFetchShowBRIE(c *C) { info2Res := brieTaskInfoToResult(info2) globalBRIEQueue.registerTask(ctx, info2) globalBRIEQueue.clearTask(e.ctx.GetSessionVars().StmtCtx) - c.Assert(fetchShowBRIEResult(c, e, brieColTypes), Equals, info2Res) + require.Equal(t, info2Res, fetchShowBRIEResult(t, e, brieColTypes)) } diff --git a/executor/builder.go b/executor/builder.go index 7aee99715c807..91288d7a25d4a 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -22,6 +22,7 @@ import ( "strconv" "strings" "sync" + "sync/atomic" "time" "unsafe" @@ -57,6 +58,7 @@ import ( "github.com/pingcap/tidb/util/cteutil" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tidb/util/timeutil" @@ -1529,6 +1531,14 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo extractor: v.Extractor.(*plannercore.ClusterLogTableExtractor), }, } + case strings.ToLower(infoschema.TableTiDBHotRegionsHistory): + return &MemTableReaderExec{ + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), + table: v.Table, + retriever: &hotRegionsHistoryRetriver{ + extractor: v.Extractor.(*plannercore.HotRegionsHistoryTableExtractor), + }, + } case strings.ToLower(infoschema.TableInspectionResult): return &MemTableReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), @@ -2770,6 +2780,7 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin) indexRanges: v.Ranges, keyOff2IdxOff: v.KeyOff2IdxOff, lastColHelper: v.CompareFilters, + finished: &atomic.Value{}, } childrenUsedSchema := markChildrenUsedCols(v.Schema(), v.Children()[0].Schema(), v.Children()[1].Schema()) e.joiner = newJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, v.OtherConditions, leftTypes, rightTypes, childrenUsedSchema) @@ -3597,21 +3608,21 @@ type mockPhysicalIndexReader struct { } func (builder *dataReaderBuilder) buildExecutorForIndexJoin(ctx context.Context, lookUpContents []*indexJoinLookUpContent, - IndexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager, canReorderHandles bool) (Executor, error) { - return builder.buildExecutorForIndexJoinInternal(ctx, builder.Plan, lookUpContents, IndexRanges, keyOff2IdxOff, cwc, canReorderHandles) + IndexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager, canReorderHandles bool, memTracker *memory.Tracker, interruptSignal *atomic.Value) (Executor, error) { + return builder.buildExecutorForIndexJoinInternal(ctx, builder.Plan, lookUpContents, IndexRanges, keyOff2IdxOff, cwc, canReorderHandles, memTracker, interruptSignal) } func (builder *dataReaderBuilder) buildExecutorForIndexJoinInternal(ctx context.Context, plan plannercore.Plan, lookUpContents []*indexJoinLookUpContent, - IndexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager, canReorderHandles bool) (Executor, error) { + IndexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager, canReorderHandles bool, memTracker *memory.Tracker, interruptSignal *atomic.Value) (Executor, error) { switch v := plan.(type) { case *plannercore.PhysicalTableReader: - return builder.buildTableReaderForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc, canReorderHandles) + return builder.buildTableReaderForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc, canReorderHandles, memTracker, interruptSignal) case *plannercore.PhysicalIndexReader: - return builder.buildIndexReaderForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc) + return builder.buildIndexReaderForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc, memTracker, interruptSignal) case *plannercore.PhysicalIndexLookUpReader: - return builder.buildIndexLookUpReaderForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc) + return builder.buildIndexLookUpReaderForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc, memTracker, interruptSignal) case *plannercore.PhysicalUnionScan: - return builder.buildUnionScanForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc, canReorderHandles) + return builder.buildUnionScanForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc, canReorderHandles, memTracker, interruptSignal) // The inner child of IndexJoin might be Projection when a combination of the following conditions is true: // 1. The inner child fetch data using indexLookupReader // 2. PK is not handle @@ -3619,11 +3630,11 @@ func (builder *dataReaderBuilder) buildExecutorForIndexJoinInternal(ctx context. // In this case, an extra column tidb_rowid will be appended in the output result of IndexLookupReader(see copTask.doubleReadNeedProj). // Then we need a Projection upon IndexLookupReader to prune the redundant column. case *plannercore.PhysicalProjection: - return builder.buildProjectionForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc) + return builder.buildProjectionForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc, memTracker, interruptSignal) // Need to support physical selection because after PR 16389, TiDB will push down all the expr supported by TiKV or TiFlash // in predicate push down stage, so if there is an expr which only supported by TiFlash, a physical selection will be added after index read case *plannercore.PhysicalSelection: - childExec, err := builder.buildExecutorForIndexJoinInternal(ctx, v.Children()[0], lookUpContents, IndexRanges, keyOff2IdxOff, cwc, canReorderHandles) + childExec, err := builder.buildExecutorForIndexJoinInternal(ctx, v.Children()[0], lookUpContents, IndexRanges, keyOff2IdxOff, cwc, canReorderHandles, memTracker, interruptSignal) if err != nil { return nil, err } @@ -3641,9 +3652,9 @@ func (builder *dataReaderBuilder) buildExecutorForIndexJoinInternal(ctx context. func (builder *dataReaderBuilder) buildUnionScanForIndexJoin(ctx context.Context, v *plannercore.PhysicalUnionScan, values []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, - cwc *plannercore.ColWithCmpFuncManager, canReorderHandles bool) (Executor, error) { + cwc *plannercore.ColWithCmpFuncManager, canReorderHandles bool, memTracker *memory.Tracker, interruptSignal *atomic.Value) (Executor, error) { childBuilder := &dataReaderBuilder{Plan: v.Children()[0], executorBuilder: builder.executorBuilder} - reader, err := childBuilder.buildExecutorForIndexJoin(ctx, values, indexRanges, keyOff2IdxOff, cwc, canReorderHandles) + reader, err := childBuilder.buildExecutorForIndexJoin(ctx, values, indexRanges, keyOff2IdxOff, cwc, canReorderHandles, memTracker, interruptSignal) if err != nil { return nil, err } @@ -3657,7 +3668,7 @@ func (builder *dataReaderBuilder) buildUnionScanForIndexJoin(ctx context.Context func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalTableReader, lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, - cwc *plannercore.ColWithCmpFuncManager, canReorderHandles bool) (Executor, error) { + cwc *plannercore.ColWithCmpFuncManager, canReorderHandles bool, memTracker *memory.Tracker, interruptSignal *atomic.Value) (Executor, error) { e, err := buildNoRangeTableReader(builder.executorBuilder, v) if err != nil { return nil, err @@ -3665,7 +3676,7 @@ func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Conte tbInfo := e.table.Meta() if v.IsCommonHandle { if tbInfo.GetPartitionInfo() == nil || !builder.ctx.GetSessionVars().UseDynamicPartitionPrune() { - kvRanges, err := buildKvRangesForIndexJoin(e.ctx, getPhysicalTableID(e.table), -1, lookUpContents, indexRanges, keyOff2IdxOff, cwc) + kvRanges, err := buildKvRangesForIndexJoin(e.ctx, getPhysicalTableID(e.table), -1, lookUpContents, indexRanges, keyOff2IdxOff, cwc, memTracker, interruptSignal) if err != nil { return nil, err } @@ -3694,7 +3705,7 @@ func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Conte return nil, err } pid := p.GetPhysicalID() - tmp, err := buildKvRangesForIndexJoin(e.ctx, pid, -1, []*indexJoinLookUpContent{content}, indexRanges, keyOff2IdxOff, cwc) + tmp, err := buildKvRangesForIndexJoin(e.ctx, pid, -1, []*indexJoinLookUpContent{content}, indexRanges, keyOff2IdxOff, cwc, nil, interruptSignal) if err != nil { return nil, err } @@ -3709,7 +3720,7 @@ func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Conte kvRanges = make([]kv.KeyRange, 0, len(partitions)*len(lookUpContents)) for _, p := range partitions { pid := p.GetPhysicalID() - tmp, err := buildKvRangesForIndexJoin(e.ctx, pid, -1, lookUpContents, indexRanges, keyOff2IdxOff, cwc) + tmp, err := buildKvRangesForIndexJoin(e.ctx, pid, -1, lookUpContents, indexRanges, keyOff2IdxOff, cwc, memTracker, interruptSignal) if err != nil { return nil, err } @@ -3875,14 +3886,14 @@ func (builder *dataReaderBuilder) buildTableReaderFromKvRanges(ctx context.Conte } func (builder *dataReaderBuilder) buildIndexReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalIndexReader, - lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) { + lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager, memoryTracker *memory.Tracker, interruptSignal *atomic.Value) (Executor, error) { e, err := buildNoRangeIndexReader(builder.executorBuilder, v) if err != nil { return nil, err } tbInfo := e.table.Meta() if tbInfo.GetPartitionInfo() == nil || !builder.ctx.GetSessionVars().UseDynamicPartitionPrune() { - kvRanges, err := buildKvRangesForIndexJoin(e.ctx, e.physicalTableID, e.index.ID, lookUpContents, indexRanges, keyOff2IdxOff, cwc) + kvRanges, err := buildKvRangesForIndexJoin(e.ctx, e.physicalTableID, e.index.ID, lookUpContents, indexRanges, keyOff2IdxOff, cwc, memoryTracker, interruptSignal) if err != nil { return nil, err } @@ -3921,7 +3932,7 @@ func (builder *dataReaderBuilder) buildIndexReaderForIndexJoin(ctx context.Conte } func (builder *dataReaderBuilder) buildIndexLookUpReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalIndexLookUpReader, - lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) { + lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager, memTracker *memory.Tracker, interruptSignal *atomic.Value) (Executor, error) { e, err := buildNoRangeIndexLookUpReader(builder.executorBuilder, v) if err != nil { return nil, err @@ -3929,7 +3940,7 @@ func (builder *dataReaderBuilder) buildIndexLookUpReaderForIndexJoin(ctx context tbInfo := e.table.Meta() if tbInfo.GetPartitionInfo() == nil || !builder.ctx.GetSessionVars().UseDynamicPartitionPrune() { - e.kvRanges, err = buildKvRangesForIndexJoin(e.ctx, getPhysicalTableID(e.table), e.index.ID, lookUpContents, indexRanges, keyOff2IdxOff, cwc) + e.kvRanges, err = buildKvRangesForIndexJoin(e.ctx, getPhysicalTableID(e.table), e.index.ID, lookUpContents, indexRanges, keyOff2IdxOff, cwc, memTracker, interruptSignal) if err != nil { return nil, err } @@ -3969,18 +3980,18 @@ func (builder *dataReaderBuilder) buildIndexLookUpReaderForIndexJoin(ctx context } func (builder *dataReaderBuilder) buildProjectionForIndexJoin(ctx context.Context, v *plannercore.PhysicalProjection, - lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) { + lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager, memTracker *memory.Tracker, interruptSignal *atomic.Value) (Executor, error) { var ( childExec Executor err error ) switch op := v.Children()[0].(type) { case *plannercore.PhysicalIndexLookUpReader: - if childExec, err = builder.buildIndexLookUpReaderForIndexJoin(ctx, op, lookUpContents, indexRanges, keyOff2IdxOff, cwc); err != nil { + if childExec, err = builder.buildIndexLookUpReaderForIndexJoin(ctx, op, lookUpContents, indexRanges, keyOff2IdxOff, cwc, memTracker, interruptSignal); err != nil { return nil, err } case *plannercore.PhysicalTableReader: - if childExec, err = builder.buildTableReaderForIndexJoin(ctx, op, lookUpContents, indexRanges, keyOff2IdxOff, cwc, true); err != nil { + if childExec, err = builder.buildTableReaderForIndexJoin(ctx, op, lookUpContents, indexRanges, keyOff2IdxOff, cwc, true, memTracker, interruptSignal); err != nil { return nil, err } default: @@ -4044,12 +4055,12 @@ func buildRangesForIndexJoin(ctx sessionctx.Context, lookUpContents []*indexJoin return retRanges, nil } - return ranger.UnionRanges(ctx.GetSessionVars().StmtCtx, tmpDatumRanges, true) + return ranger.UnionRanges(ctx, tmpDatumRanges, true) } // buildKvRangesForIndexJoin builds kv ranges for index join when the inner plan is index scan plan. func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, lookUpContents []*indexJoinLookUpContent, - ranges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (_ []kv.KeyRange, err error) { + ranges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager, memTracker *memory.Tracker, interruptSignal *atomic.Value) (_ []kv.KeyRange, err error) { kvRanges := make([]kv.KeyRange, 0, len(ranges)*len(lookUpContents)) lastPos := len(ranges[0].LowVal) - 1 sc := ctx.GetSessionVars().StmtCtx @@ -4068,7 +4079,7 @@ func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, l if indexID == -1 { tmpKvRanges, err = distsql.CommonHandleRangesToKVRanges(sc, []int64{tableID}, ranges) } else { - tmpKvRanges, err = distsql.IndexRangesToKVRanges(sc, tableID, indexID, ranges, nil) + tmpKvRanges, err = distsql.IndexRangesToKVRangesWithInterruptSignal(sc, tableID, indexID, ranges, nil, memTracker, interruptSignal) } if err != nil { return nil, err @@ -4090,7 +4101,12 @@ func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, l } } } - + if len(kvRanges) != 0 && memTracker != nil { + memTracker.Consume(int64(2 * cap(kvRanges[0].StartKey) * len(kvRanges))) + } + if len(tmpDatumRanges) != 0 && memTracker != nil { + memTracker.Consume(2 * int64(len(tmpDatumRanges)) * types.EstimatedMemUsage(tmpDatumRanges[0].LowVal, len(tmpDatumRanges))) + } if cwc == nil { sort.Slice(kvRanges, func(i, j int) bool { return bytes.Compare(kvRanges[i].StartKey, kvRanges[j].StartKey) < 0 @@ -4098,7 +4114,7 @@ func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, l return kvRanges, nil } - tmpDatumRanges, err = ranger.UnionRanges(ctx.GetSessionVars().StmtCtx, tmpDatumRanges, true) + tmpDatumRanges, err = ranger.UnionRanges(ctx, tmpDatumRanges, true) if err != nil { return nil, err } @@ -4106,7 +4122,7 @@ func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, l if indexID == -1 { return distsql.CommonHandleRangesToKVRanges(ctx.GetSessionVars().StmtCtx, []int64{tableID}, tmpDatumRanges) } - return distsql.IndexRangesToKVRanges(ctx.GetSessionVars().StmtCtx, tableID, indexID, tmpDatumRanges, nil) + return distsql.IndexRangesToKVRangesWithInterruptSignal(ctx.GetSessionVars().StmtCtx, tableID, indexID, tmpDatumRanges, nil, memTracker, interruptSignal) } func (b *executorBuilder) buildWindow(v *plannercore.PhysicalWindow) Executor { @@ -4252,7 +4268,8 @@ func (b *executorBuilder) buildShuffle(v *plannercore.PhysicalShuffle) *ShuffleE for j, dataSource := range v.DataSources { stub := plannercore.PhysicalShuffleReceiverStub{ - Receiver: (unsafe.Pointer)(receivers[j]), + Receiver: (unsafe.Pointer)(receivers[j]), + DataSource: dataSource, }.Init(b.ctx, dataSource.Stats(), dataSource.SelectBlockOffset(), nil) stub.SetSchema(dataSource.Schema()) v.Tails[j].SetChildren(stub) diff --git a/executor/checksum.go b/executor/checksum.go index 611510cec864d..69fd6ed319e75 100644 --- a/executor/checksum.go +++ b/executor/checksum.go @@ -241,7 +241,7 @@ func (c *checksumContext) buildTableRequest(ctx sessionctx.Context, tableID int6 } var builder distsql.RequestBuilder - builder.SetResourceGroupTag(ctx.GetSessionVars().StmtCtx) + builder.SetResourceGroupTagger(ctx.GetSessionVars().StmtCtx) return builder.SetHandleRanges(ctx.GetSessionVars().StmtCtx, tableID, c.TableInfo.IsCommonHandle, ranges, nil). SetChecksumRequest(checksum). SetStartTS(c.StartTs). @@ -258,7 +258,7 @@ func (c *checksumContext) buildIndexRequest(ctx sessionctx.Context, tableID int6 ranges := ranger.FullRange() var builder distsql.RequestBuilder - builder.SetResourceGroupTag(ctx.GetSessionVars().StmtCtx) + builder.SetResourceGroupTagger(ctx.GetSessionVars().StmtCtx) return builder.SetIndexRanges(ctx.GetSessionVars().StmtCtx, tableID, indexInfo.ID, ranges). SetChecksumRequest(checksum). SetStartTS(c.StartTs). diff --git a/executor/compiler.go b/executor/compiler.go index 7a519dae6fae8..74a878b4d3293 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -61,7 +61,6 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm if err != nil { return nil, err } - stmtNode = plannercore.TryAddExtraLimit(c.Ctx, stmtNode) finalPlan, names, err := planner.Optimize(ctx, c.Ctx, stmtNode, ret.InfoSchema) if err != nil { diff --git a/executor/coprocessor.go b/executor/coprocessor.go index 6eb438d5aaeb5..4d8595640d6fd 100644 --- a/executor/coprocessor.go +++ b/executor/coprocessor.go @@ -118,8 +118,8 @@ func (h *CoprocessorDAGHandler) buildResponseAndSendToStream(chk *chunk.Chunk, t return stream.Send(h.buildErrorResponse(err)) } - for _, c := range chunks { - resp := h.buildStreamResponse(&c) + for i := range chunks { + resp := h.buildStreamResponse(&chunks[i]) if err = stream.Send(resp); err != nil { return err } diff --git a/executor/ddl.go b/executor/ddl.go index ffd89234bd9c0..4c2be5828b8a8 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -185,7 +185,18 @@ func (e *DDLExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { case *ast.AlterSequenceStmt: err = e.executeAlterSequence(x) case *ast.CreatePlacementPolicyStmt: + if x.OrReplace && x.IfNotExists { + err = ddl.ErrWrongUsage.GenWithStackByArgs("OR REPLACE", "IF NOT EXISTS") + break + } err = e.executeCreatePlacementPolicy(x) + if x.OrReplace && errors.ErrorEqual(err, infoschema.ErrPlacementPolicyExists) { + alterStmt := &ast.AlterPlacementPolicyStmt{ + PolicyName: x.PolicyName, + PlacementOptions: x.PlacementOptions, + } + err = e.executeAlterPlacementPolicy(alterStmt) + } case *ast.DropPlacementPolicyStmt: err = e.executeDropPlacementPolicy(x) case *ast.AlterPlacementPolicyStmt: diff --git a/executor/distsql.go b/executor/distsql.go index 899d2e822f176..3edb1dd709168 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -374,6 +374,9 @@ type IndexLookUpExecutor struct { // extraPIDColumnIndex is used for partition reader to add an extra partition ID column, default -1 extraPIDColumnIndex offsetOptional + + // cancelFunc is called when close the executor + cancelFunc context.CancelFunc } type getHandleType int8 @@ -487,6 +490,8 @@ func (e *IndexLookUpExecutor) open(ctx context.Context) error { func (e *IndexLookUpExecutor) startWorkers(ctx context.Context, initBatchSize int) error { // indexWorker will write to workCh and tableWorker will read from workCh, // so fetching index and getting table data can run concurrently. + ctx, cancel := context.WithCancel(ctx) + e.cancelFunc = cancel workCh := make(chan *lookupTableTask, 1) if err := e.startIndexWorker(ctx, workCh, initBatchSize); err != nil { return err @@ -676,6 +681,10 @@ func (e *IndexLookUpExecutor) Close() error { return nil } + if e.cancelFunc != nil { + e.cancelFunc() + e.cancelFunc = nil + } close(e.finished) // Drain the resultCh and discard the result, in case that Next() doesn't fully // consume the data, background worker still writing to resultCh and block forever. @@ -1119,6 +1128,13 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta chk := newFirstChunk(tableReader) tblInfo := w.idxLookup.table.Meta() vals := make([]types.Datum, 0, len(w.idxTblCols)) + + // Prepare collator for compare. + collators := make([]collate.Collator, 0, len(w.idxColTps)) + for _, tp := range w.idxColTps { + collators = append(collators, collate.GetCollator(tp.Collate)) + } + for { err := Next(ctx, tableReader, chk) if err != nil { @@ -1155,19 +1171,19 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta } tablecodec.TruncateIndexValues(tblInfo, w.idxLookup.index, vals) sctx := w.idxLookup.ctx.GetSessionVars().StmtCtx - for i, val := range vals { + for i := range vals { col := w.idxTblCols[i] tp := &col.FieldType idxVal := idxRow.GetDatum(i, tp) tablecodec.TruncateIndexValue(&idxVal, w.idxLookup.index.Columns[i], col.ColumnInfo) - cmpRes, err := idxVal.CompareDatum(sctx, &val) + cmpRes, err := idxVal.Compare(sctx, &vals[i], collators[i]) if err != nil { return ErrDataInConsistentMisMatchIndex.GenWithStackByArgs(col.Name, - handle, idxRow.GetDatum(i, tp), val, err) + handle, idxRow.GetDatum(i, tp), vals[i], err) } if cmpRes != 0 { return ErrDataInConsistentMisMatchIndex.GenWithStackByArgs(col.Name, - handle, idxRow.GetDatum(i, tp), val, err) + handle, idxRow.GetDatum(i, tp), vals[i], err) } } } diff --git a/executor/executor.go b/executor/executor.go index 7383941acd046..9ade86612b96f 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -31,6 +31,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/domain/infosync" @@ -39,7 +40,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/model" @@ -968,18 +968,25 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { } func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.LockCtx { - var planDigest *parser.Digest - _, sqlDigest := seVars.StmtCtx.SQLDigest() - if variable.TopSQLEnabled() { - _, planDigest = seVars.StmtCtx.GetPlanDigest() - } lockCtx := tikvstore.NewLockCtx(seVars.TxnCtx.GetForUpdateTS(), lockWaitTime, seVars.StmtCtx.GetLockWaitStartTime()) lockCtx.Killed = &seVars.Killed lockCtx.PessimisticLockWaited = &seVars.StmtCtx.PessimisticLockWaited lockCtx.LockKeysDuration = &seVars.StmtCtx.LockKeysDuration lockCtx.LockKeysCount = &seVars.StmtCtx.LockKeysCount lockCtx.LockExpired = &seVars.TxnCtx.LockExpire - lockCtx.ResourceGroupTag = resourcegrouptag.EncodeResourceGroupTag(sqlDigest, planDigest) + lockCtx.ResourceGroupTagger = func(req *kvrpcpb.PessimisticLockRequest) []byte { + if req == nil { + return nil + } + if len(req.Mutations) == 0 { + return nil + } + if mutation := req.Mutations[0]; mutation != nil { + label := resourcegrouptag.GetResourceGroupLabelByKey(mutation.Key) + return seVars.StmtCtx.GetResourceGroupTagByLabel(label) + } + return nil + } lockCtx.OnDeadlock = func(deadlock *tikverr.ErrDeadlock) { cfg := config.GetGlobalConfig() if deadlock.IsRetryable && !cfg.PessimisticTxn.DeadlockHistoryCollectRetryable { @@ -1896,8 +1903,8 @@ func FillVirtualColumnValue(virtualRetTypes []*types.FieldType, virtualColumnInd return nil } -func setResourceGroupTagForTxn(sc *stmtctx.StatementContext, snapshot kv.Snapshot) { +func setResourceGroupTaggerForTxn(sc *stmtctx.StatementContext, snapshot kv.Snapshot) { if snapshot != nil && variable.TopSQLEnabled() { - snapshot.SetOption(kv.ResourceGroupTag, sc.GetResourceGroupTag()) + snapshot.SetOption(kv.ResourceGroupTagger, sc.GetResourceGroupTagger()) } } diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index 3fca4be179169..dc3e68acaa4f5 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -195,7 +195,7 @@ func SubTestBuildKvRangesForIndexJoinWithoutCwc(t *testing.T) { keyOff2IdxOff := []int{1, 3} ctx := mock.NewContext() - kvRanges, err := buildKvRangesForIndexJoin(ctx, 0, 0, joinKeyRows, indexRanges, keyOff2IdxOff, nil) + kvRanges, err := buildKvRangesForIndexJoin(ctx, 0, 0, joinKeyRows, indexRanges, keyOff2IdxOff, nil, nil, nil) require.NoError(t, err) // Check the kvRanges is in order. for i, kvRange := range kvRanges { diff --git a/executor/executor_test.go b/executor/executor_test.go index 1cb3f94a7765a..0c77367e007a2 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -20,6 +20,7 @@ import ( "flag" "fmt" "math" + "math/rand" "net" "os" "path/filepath" @@ -66,6 +67,7 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" + testkit2 "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" @@ -81,6 +83,7 @@ import ( "github.com/pingcap/tidb/util/testutil" "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-tipb" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/testutils" "github.com/tikv/client-go/v2/tikv" @@ -109,12 +112,9 @@ var _ = Suite(&testSuiteJoin1{&baseTestSuite{}}) var _ = Suite(&testSuiteJoin2{&baseTestSuite{}}) var _ = Suite(&testSuiteJoin3{&baseTestSuite{}}) var _ = SerialSuites(&testSuiteJoinSerial{&baseTestSuite{}}) -var _ = Suite(&testSuiteAgg{baseTestSuite: &baseTestSuite{}}) var _ = Suite(&testSuite6{&baseTestSuite{}}) var _ = Suite(&testSuite7{&baseTestSuite{}}) var _ = Suite(&testSuite8{&baseTestSuite{}}) -var _ = SerialSuites(&testShowStatsSuite{&baseTestSuite{}}) -var _ = Suite(&testBypassSuite{}) var _ = Suite(&testUpdateSuite{}) var _ = Suite(&testPointGetSuite{}) var _ = SerialSuites(&testRecoverTable{}) @@ -588,6 +588,41 @@ func (s *testSuiteP2) TestAdminShowDDLJobs(c *C) { c.Assert(row[9], Equals, "") } +func (s *testSuiteP2) TestAdminShowDDLJobsInfo(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test_admin_show_ddl_jobs") + defer tk.MustExec("drop database if exists test_admin_show_ddl_jobs") + tk.MustExec("use test_admin_show_ddl_jobs") + tk.MustExec("drop table if exists t, t1;") + tk.MustExec("create table t (a int);") + tk.MustExec("create table t1 (a int);") + + // Test for issue: https://github.com/pingcap/tidb/issues/29915 + tk.MustExec("drop placement policy if exists x;") + tk.MustExec("create placement policy x followers=4;") + tk.MustExec("alter table t placement policy x;") + c.Assert(tk.MustQuery("admin show ddl jobs 1").Rows()[0][3], Equals, "alter table placement") + + tk.MustExec("rename table t to tt, t1 to tt1") + c.Assert(tk.MustQuery("admin show ddl jobs 1").Rows()[0][3], Equals, "rename tables") + + tk.MustExec("create table tt2 (c int) PARTITION BY RANGE (c) " + + "(PARTITION p0 VALUES LESS THAN (6)," + + "PARTITION p1 VALUES LESS THAN (11)," + + "PARTITION p2 VALUES LESS THAN (16)," + + "PARTITION p3 VALUES LESS THAN (21));") + tk.MustExec("alter table tt2 partition p0 " + + "PRIMARY_REGION=\"cn-east-1\" " + + "REGIONS=\"cn-east-1, cn-east-2\" " + + "FOLLOWERS=2 ") + c.Assert(tk.MustQuery("admin show ddl jobs 1").Rows()[0][3], Equals, "alter table partition policy") + + tk.MustExec("alter table tt1 cache") + c.Assert(tk.MustQuery("admin show ddl jobs 1").Rows()[0][3], Equals, "alter table cache") + tk.MustExec("alter table tt1 nocache") + c.Assert(tk.MustQuery("admin show ddl jobs 1").Rows()[0][3], Equals, "alter table nocache") +} + func (s *testSuiteP2) TestAdminChecksumOfPartitionedTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("USE test;") @@ -619,35 +654,33 @@ type testCase struct { } func checkCases(tests []testCase, ld *executor.LoadDataInfo, - c *C, tk *testkit.TestKit, ctx sessionctx.Context, selectSQL, deleteSQL string) { + t *testing.T, tk *testkit2.TestKit, ctx sessionctx.Context, selectSQL, deleteSQL string) { origin := ld.IgnoreLines for _, tt := range tests { ld.IgnoreLines = origin - c.Assert(ctx.NewTxn(context.Background()), IsNil) + require.Nil(t, ctx.NewTxn(context.Background())) ctx.GetSessionVars().StmtCtx.DupKeyAsWarning = true ctx.GetSessionVars().StmtCtx.BadNullAsWarning = true ctx.GetSessionVars().StmtCtx.InLoadDataStmt = true ctx.GetSessionVars().StmtCtx.InDeleteStmt = false data, reachLimit, err1 := ld.InsertData(context.Background(), tt.data1, tt.data2) - c.Assert(err1, IsNil) - c.Assert(reachLimit, IsFalse) + require.NoError(t, err1) + require.False(t, reachLimit) err1 = ld.CheckAndInsertOneBatch(context.Background(), ld.GetRows(), ld.GetCurBatchCnt()) - c.Assert(err1, IsNil) + require.NoError(t, err1) ld.SetMaxRowsInBatch(20000) if tt.restData == nil { - c.Assert(data, HasLen, 0, - Commentf("data1:%v, data2:%v, data:%v", string(tt.data1), string(tt.data2), string(data))) + require.Len(t, data, 0, "data1:%v, data2:%v, data:%v", string(tt.data1), string(tt.data2), string(data)) } else { - c.Assert(data, DeepEquals, tt.restData, - Commentf("data1:%v, data2:%v, data:%v", string(tt.data1), string(tt.data2), string(data))) + require.Equal(t, tt.restData, data, "data1:%v, data2:%v, data:%v", string(tt.data1), string(tt.data2), string(data)) } ld.SetMessage() - tk.CheckLastMessage(tt.expectedMsg) + require.Equal(t, tt.expectedMsg, tk.Session().LastMessage()) ctx.StmtCommit() txn, err := ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) r := tk.MustQuery(selectSQL) r.Check(testutil.RowsWithSep("|", tt.expected...)) tk.MustExec(deleteSQL) @@ -3315,6 +3348,16 @@ func (s *testSuite) TestEmptyEnum(c *C) { tk.MustQuery("select * from t").Check(testkit.Rows("", "")) tk.MustExec("insert into t values (null)") tk.MustQuery("select * from t").Check(testkit.Rows("", "", "")) + + // Test https://github.com/pingcap/tidb/issues/29525. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (id int auto_increment primary key, c1 enum('a', '', 'c'));") + tk.MustExec("insert into t(c1) values (0);") + tk.MustQuery("select id, c1+0, c1 from t;").Check(testkit.Rows("1 0 ")) + tk.MustExec("alter table t change c1 c1 enum('a', '') not null;") + tk.MustQuery("select id, c1+0, c1 from t;").Check(testkit.Rows("1 0 ")) + tk.MustExec("insert into t(c1) values (0);") + tk.MustQuery("select id, c1+0, c1 from t;").Check(testkit.Rows("1 0 ", "2 0 ")) } // TestIssue4024 This tests https://github.com/pingcap/tidb/issues/4024 @@ -8685,6 +8728,7 @@ func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { defer failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook") var sqlDigest, planDigest *parser.Digest + var tagLabel tipb.ResourceGroupTagLabel checkFn := func() {} unistore.UnistoreRPCClientSendHook = func(req *tikvrpc.Request) { var startKey []byte @@ -8727,6 +8771,7 @@ func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { c.Assert(err, IsNil) sqlDigest = parser.NewDigest(tag.SqlDigest) planDigest = parser.NewDigest(tag.PlanDigest) + tagLabel = *tag.Label checkFn() } @@ -8736,19 +8781,78 @@ func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { } cases := []struct { - sql string - ignore bool + sql string + tagLabels []tipb.ResourceGroupTagLabel + ignore bool }{ - {sql: "insert into t values(1,1),(2,2),(3,3)"}, - {sql: "select * from t use index (idx) where a=1"}, - {sql: "select * from t use index (idx) where a in (1,2,3)"}, - {sql: "select * from t use index (idx) where a>1"}, - {sql: "select * from t where b>1"}, - {sql: "begin pessimistic", ignore: true}, - {sql: "insert into t values(4,4)"}, - {sql: "commit", ignore: true}, - {sql: "update t set a=5,b=5 where a=5"}, - {sql: "replace into t values(6,6)"}, + { + sql: "insert into t values(1,1),(2,2),(3,3)", + tagLabels: []tipb.ResourceGroupTagLabel{ + tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex, + tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex, + }, + }, + { + sql: "select * from t use index (idx) where a=1", + tagLabels: []tipb.ResourceGroupTagLabel{ + tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex, + tipb.ResourceGroupTagLabel_ResourceGroupTagLabelRow, + }, + }, + { + sql: "select * from t use index (idx) where a in (1,2,3)", + tagLabels: []tipb.ResourceGroupTagLabel{ + tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex, + tipb.ResourceGroupTagLabel_ResourceGroupTagLabelRow, + }, + }, + { + sql: "select * from t use index (idx) where a>1", + tagLabels: []tipb.ResourceGroupTagLabel{ + tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex, + tipb.ResourceGroupTagLabel_ResourceGroupTagLabelRow, + }, + }, + { + sql: "select * from t where b>1", + tagLabels: []tipb.ResourceGroupTagLabel{ + tipb.ResourceGroupTagLabel_ResourceGroupTagLabelRow, + }, + }, + { + sql: "select a from t use index (idx) where a>1", + tagLabels: []tipb.ResourceGroupTagLabel{ + tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex, + }, + }, + { + sql: "begin pessimistic", + ignore: true, + }, + { + sql: "insert into t values(4,4)", + tagLabels: []tipb.ResourceGroupTagLabel{ + tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex, + tipb.ResourceGroupTagLabel_ResourceGroupTagLabelRow, + }, + }, + { + sql: "commit", + ignore: true, + }, + { + sql: "update t set a=5,b=5 where a=5", + tagLabels: []tipb.ResourceGroupTagLabel{ + tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex, + }, + }, + { + sql: "replace into t values(6,6)", + tagLabels: []tipb.ResourceGroupTagLabel{ + tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex, + tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex, + }, + }, } for _, ca := range cases { resetVars() @@ -8770,6 +8874,10 @@ func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { } c.Assert(sqlDigest.String(), Equals, expectSQLDigest.String(), commentf) c.Assert(planDigest.String(), Equals, expectPlanDigest.String()) + if len(ca.tagLabels) > 0 { + c.Assert(tagLabel, Equals, ca.tagLabels[0]) + ca.tagLabels = ca.tagLabels[1:] // next label + } checkCnt++ } @@ -9325,3 +9433,61 @@ func (s *testSuiteP1) TestIssue29412(c *C) { tk.MustExec("insert into t29142_1 value(20);") tk.MustQuery("select sum(distinct a) as x from t29142_1 having x > some ( select a from t29142_2 where x in (a));").Check(nil) } + +func (s *testSerialSuite) TestIssue28650(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("create table t1(a int, index(a));") + tk.MustExec("create table t2(a int, c int, b char(50), index(a,c,b));") + tk.MustExec("set tidb_enable_rate_limit_action=off;") + + wg := &sync.WaitGroup{} + sql := `explain analyze + select /*+ stream_agg(@sel_1) stream_agg(@sel_3) %s(@sel_2 t2)*/ count(1) from + ( + SELECT t2.a AS t2_external_user_ext_id, t2.b AS t2_t1_ext_id FROM t2 INNER JOIN (SELECT t1.a AS d_t1_ext_id FROM t1 GROUP BY t1.a) AS anon_1 ON anon_1.d_t1_ext_id = t2.a WHERE t2.c = 123 AND t2.b + IN ("%s") ) tmp` + + wg.Add(1) + sqls := make([]string, 2) + go func() { + defer wg.Done() + inElems := make([]string, 1000) + for i := 0; i < len(inElems); i++ { + inElems[i] = fmt.Sprintf("wm_%dbDgAAwCD-v1QB%dxky-g_dxxQCw", rand.Intn(100), rand.Intn(100)) + } + sqls[0] = fmt.Sprintf(sql, "inl_join", strings.Join(inElems, "\",\"")) + sqls[1] = fmt.Sprintf(sql, "inl_hash_join", strings.Join(inElems, "\",\"")) + }() + + tk.MustExec("insert into t1 select rand()*400;") + for i := 0; i < 10; i++ { + tk.MustExec("insert into t1 select rand()*400 from t1;") + } + config.UpdateGlobal(func(conf *config.Config) { + conf.OOMAction = config.OOMActionCancel + }) + defer func() { + config.UpdateGlobal(func(conf *config.Config) { + conf.OOMAction = config.OOMActionLog + }) + }() + wg.Wait() + for _, sql := range sqls { + tk.MustExec("set @@tidb_mem_quota_query = 1073741824") // 1GB + c.Assert(tk.QueryToErr(sql), IsNil) + tk.MustExec("set @@tidb_mem_quota_query = 33554432") // 32MB, out of memory during executing + c.Assert(strings.Contains(tk.QueryToErr(sql).Error(), "Out Of Memory Quota!"), IsTrue) + tk.MustExec("set @@tidb_mem_quota_query = 65536") // 64KB, out of memory during building the plan + func() { + defer func() { + r := recover() + c.Assert(r, NotNil) + err := errors.Errorf("%v", r) + c.Assert(strings.Contains(err.Error(), "Out Of Memory Quota!"), IsTrue) + }() + tk.MustExec(sql) + }() + } +} diff --git a/executor/explain_test.go b/executor/explain_test.go index c5d23628dfbaa..a26f373e10b71 100644 --- a/executor/explain_test.go +++ b/executor/explain_test.go @@ -365,3 +365,24 @@ func (s *testSuite) TestExplainStatementsSummary(c *C) { tk.MustQuery("desc select * from information_schema.statements_summary where digest in ('a','b','c')").Check(testutil.RowsWithSep(" ", `MemTableScan_5 10000.00 root table:STATEMENTS_SUMMARY digests: ["a","b","c"]`)) } + +func (s *testSuite) TestFix29401(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists tt123;") + tk.MustExec(`CREATE TABLE tt123 ( + id int(11) NOT NULL, + a bigint(20) DEFAULT NULL, + b char(20) DEFAULT NULL, + c datetime DEFAULT NULL, + d double DEFAULT NULL, + e json DEFAULT NULL, + f decimal(40,6) DEFAULT NULL, + PRIMARY KEY (id) /*T![clustered_index] CLUSTERED */, + KEY a (a), + KEY b (b), + KEY c (c), + KEY d (d), + KEY f (f) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;`) + tk.MustExec(" explain select /*+ inl_hash_join(t1) */ * from tt123 t1 join tt123 t2 on t1.b=t2.e;") +} diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index 794fdc7e84cae..e3dbfd42d3a52 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -20,7 +20,6 @@ import ( "fmt" "math" "strconv" - "strings" "sync" . "github.com/pingcap/check" @@ -514,9 +513,14 @@ func (s *testPrepareSerialSuite) TestPointGetUserVarPlanCache(c *C) { tkProcess := tk.Se.ShowProcess() ps := []*util.ProcessInfo{tkProcess} tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() - c.Assert(strings.Contains(fmt.Sprintf("%v", rows[5][0]), "IndexRangeScan"), IsTrue) - c.Assert(strings.Contains(fmt.Sprintf("%v", rows[5][3]), "table:t2"), IsTrue) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // can use idx_a + `Projection_9 1.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b`, + `└─IndexJoin_17 1.00 root inner join, inner:TableReader_13, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)`, + ` ├─Selection_44(Build) 0.80 root not(isnull(test.t2.a))`, + ` │ └─Point_Get_43 1.00 root table:t2, index:idx_a(a) `, + ` └─TableReader_13(Probe) 0.00 root data:Selection_12`, + ` └─Selection_12 0.00 cop[tikv] eq(test.t1.a, 1)`, + ` └─TableRangeScan_11 1.00 cop[tikv] table:t1 range: decided by [test.t2.a], keep order:false, stats:pseudo`)) tk.MustExec("set @a=2") tk.MustQuery("execute stmt using @a").Check(testkit.Rows( @@ -525,9 +529,14 @@ func (s *testPrepareSerialSuite) TestPointGetUserVarPlanCache(c *C) { tkProcess = tk.Se.ShowProcess() ps = []*util.ProcessInfo{tkProcess} tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() - c.Assert(strings.Contains(fmt.Sprintf("%v", rows[5][0]), "IndexRangeScan"), IsTrue) - c.Assert(strings.Contains(fmt.Sprintf("%v", rows[5][3]), "table:t2"), IsTrue) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // can use idx_a + `Projection_9 1.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b`, + `└─IndexJoin_17 1.00 root inner join, inner:TableReader_13, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)`, + ` ├─Selection_44(Build) 0.80 root not(isnull(test.t2.a))`, + ` │ └─Point_Get_43 1.00 root table:t2, index:idx_a(a) `, + ` └─TableReader_13(Probe) 0.00 root data:Selection_12`, + ` └─Selection_12 0.00 cop[tikv] eq(test.t1.a, 2)`, + ` └─TableRangeScan_11 1.00 cop[tikv] table:t1 range: decided by [test.t2.a], keep order:false, stats:pseudo`)) tk.MustQuery("execute stmt using @a").Check(testkit.Rows( "2 4 2 2", )) diff --git a/executor/grant.go b/executor/grant.go index 00cbee41123df..cbe15b2c350d0 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -72,8 +72,20 @@ func (e *GrantExec) Next(ctx context.Context, req *chunk.Chunk) error { dbName = e.ctx.GetSessionVars().CurrentDB } - // Make sure the table exist. + // For table & column level, check whether table exists and privilege is valid if e.Level.Level == ast.GrantLevelTable { + // Return if privilege is invalid, to fail before not existing table, see issue #29302 + for _, p := range e.Privs { + if len(p.Cols) == 0 { + if !mysql.AllTablePrivs.Has(p.Priv) && p.Priv != mysql.AllPriv && p.Priv != mysql.UsagePriv && p.Priv != mysql.GrantPriv && p.Priv != mysql.ExtendedPriv { + return ErrIllegalGrantForTable + } + } else { + if !mysql.AllColumnPrivs.Has(p.Priv) && p.Priv != mysql.AllPriv && p.Priv != mysql.UsagePriv { + return ErrWrongUsage.GenWithStackByArgs("COLUMN GRANT", "NON-COLUMN PRIVILEGES") + } + } + } dbNameStr := model.NewCIStr(dbName) schema := e.ctx.GetInfoSchema().(infoschema.InfoSchema) tbl, err := schema.TableByName(dbNameStr, model.NewCIStr(e.Level.TableName)) @@ -633,13 +645,6 @@ func composeDBPrivUpdate(sql *strings.Builder, priv mysql.PrivilegeType, value s func composeTablePrivUpdateForGrant(ctx sessionctx.Context, sql *strings.Builder, priv mysql.PrivilegeType, name string, host string, db string, tbl string) error { var newTablePriv, newColumnPriv []string if priv != mysql.AllPriv { - // TODO: https://github.com/pingcap/parser/pull/581 removed privs from all priv lists - // it is to avoid add GRANT in GRANT ALL SQLs - // WithGRANT seems broken, fix it later - if priv != mysql.GrantPriv && !mysql.AllTablePrivs.Has(priv) { - return ErrIllegalGrantForTable - } - currTablePriv, currColumnPriv, err := getTablePriv(ctx, name, host, db, tbl) if err != nil { return err @@ -669,10 +674,6 @@ func composeTablePrivUpdateForGrant(ctx sessionctx.Context, sql *strings.Builder func composeColumnPrivUpdateForGrant(ctx sessionctx.Context, sql *strings.Builder, priv mysql.PrivilegeType, name string, host string, db string, tbl string, col string) error { var newColumnPriv []string if priv != mysql.AllPriv { - if !mysql.AllColumnPrivs.Has(priv) { - return ErrWrongUsage.GenWithStackByArgs("COLUMN GRANT", "NON-COLUMN PRIVILEGES") - } - currColumnPriv, err := getColumnPriv(ctx, name, host, db, tbl, col) if err != nil { return err diff --git a/executor/grant_test.go b/executor/grant_test.go index 906d87e0b5cd2..a42b546e42b05 100644 --- a/executor/grant_test.go +++ b/executor/grant_test.go @@ -615,3 +615,19 @@ func TestGrantDynamicPrivs(t *testing.T) { tk.MustQuery("SELECT Grant_Priv FROM mysql.user WHERE `Host` = '%' AND `User` = 'dyn'").Check(testkit.Rows("Y")) tk.MustQuery("SELECT WITH_GRANT_OPTION FROM mysql.global_grants WHERE `Host` = '%' AND `User` = 'dyn' AND Priv='CONNECTION_ADMIN'").Check(testkit.Rows("Y")) } + +func TestNonExistTableIllegalGrant(t *testing.T) { + t.Parallel() + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("create user u29302") + defer tk.MustExec("drop user u29302") + // Table level, not existing table, illegal privilege + tk.MustGetErrCode("grant create temporary tables on NotExistsD29302.NotExistsT29302 to u29302", mysql.ErrIllegalGrantForTable) + tk.MustGetErrCode("grant lock tables on test.NotExistsT29302 to u29302", mysql.ErrIllegalGrantForTable) + // Column level, not existing table, illegal privilege + tk.MustGetErrCode("grant create temporary tables (NotExistsCol) on NotExistsD29302.NotExistsT29302 to u29302;", mysql.ErrWrongUsage) +} diff --git a/executor/index_lookup_hash_join.go b/executor/index_lookup_hash_join.go index 755c0713f13f8..20220b9950661 100644 --- a/executor/index_lookup_hash_join.go +++ b/executor/index_lookup_hash_join.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/expression" plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" @@ -152,6 +153,7 @@ func (e *IndexNestedLoopHashJoin) Open(ctx context.Context) error { e.stats = &indexLookUpJoinRuntimeStats{} e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } + e.finished.Store(false) e.startWorkers(ctx) return nil } @@ -201,6 +203,7 @@ func (e *IndexNestedLoopHashJoin) startWorkers(ctx context.Context) { func (e *IndexNestedLoopHashJoin) finishJoinWorkers(r interface{}) { if r != nil { + e.IndexLookUpJoin.finished.Store(true) err := errors.New(fmt.Sprintf("%v", r)) if !e.keepOuterOrder { e.resultCh <- &indexHashJoinResult{err: err} @@ -209,6 +212,7 @@ func (e *IndexNestedLoopHashJoin) finishJoinWorkers(r interface{}) { e.taskCh <- task } if e.cancelFunc != nil { + e.IndexLookUpJoin.ctxCancelReason.Store(err) e.cancelFunc() } } @@ -245,6 +249,9 @@ func (e *IndexNestedLoopHashJoin) Next(ctx context.Context, req *chunk.Chunk) er return result.err } case <-ctx.Done(): + if err := e.IndexLookUpJoin.ctxCancelReason.Load(); err != nil { + return err.(error) + } return ctx.Err() } req.SwapColumns(result.chk) @@ -274,6 +281,9 @@ func (e *IndexNestedLoopHashJoin) runInOrder(ctx context.Context, req *chunk.Chu return result.err } case <-ctx.Done(): + if err := e.IndexLookUpJoin.ctxCancelReason.Load(); err != nil { + return err.(error) + } return ctx.Err() } req.SwapColumns(result.chk) @@ -319,6 +329,7 @@ func (e *IndexNestedLoopHashJoin) Close() error { close(e.joinChkResourceCh[i]) } e.joinChkResourceCh = nil + e.finished.Store(false) return e.baseExecutor.Close() } @@ -432,6 +443,8 @@ func (e *IndexNestedLoopHashJoin) newInnerWorker(taskCh chan *indexHashJoinTask, indexRanges: copiedRanges, keyOff2IdxOff: e.keyOff2IdxOff, stats: innerStats, + lookup: &e.IndexLookUpJoin, + memTracker: memory.NewTracker(memory.LabelForIndexJoinInnerWorker, -1), }, taskCh: taskCh, joiner: e.joiners[workerID], @@ -441,6 +454,14 @@ func (e *IndexNestedLoopHashJoin) newInnerWorker(taskCh chan *indexHashJoinTask, joinKeyBuf: make([]byte, 1), outerRowStatus: make([]outerRowStatusFlag, 0, e.maxChunkSize), } + iw.memTracker.AttachTo(e.memTracker) + if len(copiedRanges) != 0 { + // We should not consume this memory usage in `iw.memTracker`. The + // memory usage of inner worker will be reset the end of iw.handleTask. + // While the life cycle of this memory consumption exists throughout the + // whole active period of inner worker. + e.ctx.GetSessionVars().StmtCtx.MemTracker.Consume(2 * types.EstimatedMemUsage(copiedRanges[0].LowVal, len(copiedRanges))) + } if e.lastColHelper != nil { // nextCwf.TmpConstant needs to be reset for every individual // inner worker to avoid data race when the inner workers is running @@ -584,6 +605,9 @@ func (iw *indexHashJoinInnerWorker) handleHashJoinInnerWorkerPanic(r interface{} } func (iw *indexHashJoinInnerWorker) handleTask(ctx context.Context, task *indexHashJoinTask, joinResult *indexHashJoinResult, h hash.Hash64, resultCh chan *indexHashJoinResult) error { + defer func() { + iw.memTracker.Consume(-iw.memTracker.BytesConsumed()) + }() var joinStartTime time.Time if iw.stats != nil { start := time.Now() @@ -631,6 +655,9 @@ func (iw *indexHashJoinInnerWorker) doJoinUnordered(ctx context.Context, task *i select { case resultCh <- joinResult: case <-ctx.Done(): + if err := iw.lookup.ctxCancelReason.Load(); err != nil { + return err.(error) + } return ctx.Err() } joinResult, ok = iw.getNewJoinResult(ctx) @@ -779,6 +806,9 @@ func (iw *indexHashJoinInnerWorker) doJoinInOrder(ctx context.Context, task *ind select { case resultCh <- joinResult: case <-ctx.Done(): + if err := iw.lookup.ctxCancelReason.Load(); err != nil { + return err.(error) + } return ctx.Err() } joinResult, ok = iw.getNewJoinResult(ctx) diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index ec5242d241df2..726fc597212a4 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -82,7 +82,9 @@ type IndexLookUpJoin struct { memTracker *memory.Tracker // track memory usage. - stats *indexLookUpJoinRuntimeStats + stats *indexLookUpJoinRuntimeStats + ctxCancelReason atomic.Value + finished *atomic.Value } type outerCtx struct { @@ -145,11 +147,13 @@ type innerWorker struct { outerCtx outerCtx ctx sessionctx.Context executorChk *chunk.Chunk + lookup *IndexLookUpJoin indexRanges []*ranger.Range nextColCompareFilters *plannercore.ColWithCmpFuncManager keyOff2IdxOff []int stats *innerWorkerRuntimeStats + memTracker *memory.Tracker } // Open implements the Executor interface. @@ -161,6 +165,7 @@ func (e *IndexLookUpJoin) Open(ctx context.Context) error { e.memTracker = memory.NewTracker(e.id, -1) e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) e.innerPtrBytes = make([][]byte, 0, 8) + e.finished.Store(false) if e.runtimeStats != nil { e.stats = &indexLookUpJoinRuntimeStats{} e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) @@ -222,6 +227,16 @@ func (e *IndexLookUpJoin) newInnerWorker(taskCh chan *lookUpJoinTask) *innerWork indexRanges: copiedRanges, keyOff2IdxOff: e.keyOff2IdxOff, stats: innerStats, + lookup: e, + memTracker: memory.NewTracker(memory.LabelForIndexJoinInnerWorker, -1), + } + iw.memTracker.AttachTo(e.memTracker) + if len(copiedRanges) != 0 { + // We should not consume this memory usage in `iw.memTracker`. The + // memory usage of inner worker will be reset the end of iw.handleTask. + // While the life cycle of this memory consumption exists throughout the + // whole active period of inner worker. + e.ctx.GetSessionVars().StmtCtx.MemTracker.Consume(2 * types.EstimatedMemUsage(copiedRanges[0].LowVal, len(copiedRanges))) } if e.lastColHelper != nil { // nextCwf.TmpConstant needs to be reset for every individual @@ -298,6 +313,9 @@ func (e *IndexLookUpJoin) getFinishedTask(ctx context.Context) (*lookUpJoinTask, select { case task = <-e.resultCh: case <-ctx.Done(): + if err := e.ctxCancelReason.Load(); err != nil { + return nil, err.(error) + } return nil, ctx.Err() } if task == nil { @@ -310,6 +328,9 @@ func (e *IndexLookUpJoin) getFinishedTask(ctx context.Context) (*lookUpJoinTask, return nil, err } case <-ctx.Done(): + if err := e.ctxCancelReason.Load(); err != nil { + return nil, err.(error) + } return nil, ctx.Err() } @@ -333,13 +354,16 @@ func (ow *outerWorker) run(ctx context.Context, wg *sync.WaitGroup) { defer trace.StartRegion(ctx, "IndexLookupJoinOuterWorker").End() defer func() { if r := recover(); r != nil { + ow.lookup.finished.Store(true) buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] logutil.Logger(ctx).Error("outerWorker panicked", zap.String("stack", string(buf))) task := &lookUpJoinTask{doneCh: make(chan error, 1)} - task.doneCh <- errors.Errorf("%v", r) - ow.pushToChan(ctx, task, ow.resultCh) + err := errors.Errorf("%v", r) + task.doneCh <- err + ow.lookup.ctxCancelReason.Store(err) + ow.lookup.cancelFunc() } close(ow.resultCh) close(ow.innerCh) @@ -450,12 +474,16 @@ func (iw *innerWorker) run(ctx context.Context, wg *sync.WaitGroup) { var task *lookUpJoinTask defer func() { if r := recover(); r != nil { + iw.lookup.finished.Store(true) buf := make([]byte, 4096) stackSize := runtime.Stack(buf, false) buf = buf[:stackSize] logutil.Logger(ctx).Error("innerWorker panicked", zap.String("stack", string(buf))) + err := errors.Errorf("%v", r) // "task != nil" is guaranteed when panic happened. - task.doneCh <- errors.Errorf("%v", r) + task.doneCh <- err + iw.lookup.ctxCancelReason.Store(err) + iw.lookup.cancelFunc() } wg.Done() }() @@ -489,6 +517,9 @@ func (iw *innerWorker) handleTask(ctx context.Context, task *lookUpJoinTask) err atomic.AddInt64(&iw.stats.totalTime, int64(time.Since(start))) }() } + defer func() { + iw.memTracker.Consume(-iw.memTracker.BytesConsumed()) + }() lookUpContents, err := iw.constructLookupContent(task) if err != nil { return err @@ -527,6 +558,9 @@ func (iw *innerWorker) constructLookupContent(task *lookUpJoinTask) ([]*indexJoi } return nil, err } + if rowIdx == 0 { + iw.lookup.memTracker.Consume(types.EstimatedMemUsage(dLookUpKey, numRows)) + } if dHashKey == nil { // Append null to make lookUpKeys the same length as outer Result. task.encodedLookUpKeys[chkIdx].AppendNull(0) @@ -651,7 +685,7 @@ func (iw *innerWorker) fetchInnerResults(ctx context.Context, task *lookUpJoinTa atomic.AddInt64(&iw.stats.fetch, int64(time.Since(start))) }() } - innerExec, err := iw.readerBuilder.buildExecutorForIndexJoin(ctx, lookUpContent, iw.indexRanges, iw.keyOff2IdxOff, iw.nextColCompareFilters, true) + innerExec, err := iw.readerBuilder.buildExecutorForIndexJoin(ctx, lookUpContent, iw.indexRanges, iw.keyOff2IdxOff, iw.nextColCompareFilters, true, iw.memTracker, iw.lookup.finished) if innerExec != nil { defer terror.Call(innerExec.Close) } @@ -665,6 +699,9 @@ func (iw *innerWorker) fetchInnerResults(ctx context.Context, task *lookUpJoinTa for { select { case <-ctx.Done(): + if err := iw.lookup.ctxCancelReason.Load(); err != nil { + return err.(error) + } return ctx.Err() default: } @@ -733,6 +770,7 @@ func (e *IndexLookUpJoin) Close() error { e.workerWg.Wait() e.memTracker = nil e.task = nil + e.finished.Store(false) return e.baseExecutor.Close() } diff --git a/executor/index_lookup_merge_join.go b/executor/index_lookup_merge_join.go index 9bbe55537421b..746fc6a5733fc 100644 --- a/executor/index_lookup_merge_join.go +++ b/executor/index_lookup_merge_join.go @@ -502,7 +502,7 @@ func (imw *innerMergeWorker) handleTask(ctx context.Context, task *lookUpMergeJo dLookUpKeys[i], dLookUpKeys[lenKeys-i-1] = dLookUpKeys[lenKeys-i-1], dLookUpKeys[i] } } - imw.innerExec, err = imw.readerBuilder.buildExecutorForIndexJoin(ctx, dLookUpKeys, imw.indexRanges, imw.keyOff2IdxOff, imw.nextColCompareFilters, false) + imw.innerExec, err = imw.readerBuilder.buildExecutorForIndexJoin(ctx, dLookUpKeys, imw.indexRanges, imw.keyOff2IdxOff, imw.nextColCompareFilters, false, nil, nil) if imw.innerExec != nil { defer terror.Call(imw.innerExec.Close) } diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 14e883571d0fb..483e970d479fc 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1453,13 +1453,13 @@ func (e *memtableRetriever) setDataForTiKVRegionStatus(ctx sessionctx.Context) e } allSchemas := ctx.GetInfoSchema().(infoschema.InfoSchema).AllSchemas() tableInfos := tikvHelper.GetRegionsTableInfo(regionsInfo, allSchemas) - for _, region := range regionsInfo.Regions { - tableList := tableInfos[region.ID] + for i := range regionsInfo.Regions { + tableList := tableInfos[regionsInfo.Regions[i].ID] if len(tableList) == 0 { - e.setNewTiKVRegionStatusCol(®ion, nil) + e.setNewTiKVRegionStatusCol(®ionsInfo.Regions[i], nil) } - for _, table := range tableList { - e.setNewTiKVRegionStatusCol(®ion, &table) + for j := range tableList { + e.setNewTiKVRegionStatusCol(®ionsInfo.Regions[i], &tableList[j]) } } return nil @@ -1508,8 +1508,8 @@ func (e *memtableRetriever) setDataForTikVRegionPeers(ctx sessionctx.Context) er if err != nil { return err } - for _, region := range regionsInfo.Regions { - e.setNewTiKVRegionPeersCols(®ion) + for i := range regionsInfo.Regions { + e.setNewTiKVRegionPeersCols(®ionsInfo.Regions[i]) } return nil } diff --git a/executor/insert.go b/executor/insert.go index 714964c4715c8..2862416ddf04a 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -66,7 +66,7 @@ func (e *InsertExec) exec(ctx context.Context, rows [][]types.Datum) error { if err != nil { return err } - setResourceGroupTagForTxn(sessVars.StmtCtx, txn) + setResourceGroupTaggerForTxn(sessVars.StmtCtx, txn) txnSize := txn.Size() sessVars.StmtCtx.AddRecordRows(uint64(len(rows))) // If you use the IGNORE keyword, duplicate-key error that occurs while executing the INSERT statement are ignored. diff --git a/executor/insert_common.go b/executor/insert_common.go index 57f987592fb1b..e16253a7756bc 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -783,7 +783,10 @@ func (e *InsertValues) adjustAutoIncrementDatum(ctx context.Context, d types.Dat if retryInfo.Retrying { id, ok := retryInfo.GetCurrAutoIncrementID() if ok { - d.SetAutoID(id, c.Flag) + err := setDatumAutoIDAndCast(e.ctx, &d, id, c) + if err != nil { + return types.Datum{}, err + } return d, nil } } @@ -856,7 +859,10 @@ func (e *InsertValues) adjustAutoRandomDatum(ctx context.Context, d types.Datum, if retryInfo.Retrying { autoRandomID, ok := retryInfo.GetCurrAutoRandomID() if ok { - d.SetAutoID(autoRandomID, c.Flag) + err := setDatumAutoIDAndCast(e.ctx, &d, autoRandomID, c) + if err != nil { + return types.Datum{}, err + } return d, nil } } @@ -882,7 +888,10 @@ func (e *InsertValues) adjustAutoRandomDatum(ctx context.Context, d types.Datum, return types.Datum{}, err } e.ctx.GetSessionVars().StmtCtx.InsertID = uint64(recordID) - d.SetAutoID(recordID, c.Flag) + err = setDatumAutoIDAndCast(e.ctx, &d, recordID, c) + if err != nil { + return types.Datum{}, err + } retryInfo.AddAutoRandomID(recordID) return d, nil } diff --git a/executor/insert_test.go b/executor/insert_test.go index d2d5b232b53af..6a2245abb96e3 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -331,7 +331,7 @@ func (s *testSuite3) TestInsertWrongValueForField(c *C) { tk.MustExec(`create table t1(a char(10) charset utf8);`) tk.MustExec(`insert into t1 values('我');`) tk.MustExec(`alter table t1 add column b char(10) charset ascii as ((a));`) - tk.MustQuery(`select * from t1;`).Check(testkit.Rows(`我 `)) + tk.MustQuery(`select * from t1;`).Check(testkit.Rows("我 ?")) tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t (a year);`) @@ -1753,3 +1753,70 @@ func (s *testSuite13) TestIssue26762(c *C) { _, err = tk.Exec("insert into t1 values('2020-02-31');") c.Assert(err.Error(), Equals, `[table:1292]Incorrect date value: '2020-02-31' for column 'c1' at row 1`) } + +func (s *testSuite10) TestStringtoDecimal(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (id decimal(10))") + tk.MustGetErrCode("insert into t values('1sdf')", errno.ErrTruncatedWrongValueForField) + tk.MustGetErrCode("insert into t values('1edf')", errno.ErrTruncatedWrongValueForField) + tk.MustGetErrCode("insert into t values('12Ea')", errno.ErrTruncatedWrongValueForField) + tk.MustGetErrCode("insert into t values('1E')", errno.ErrTruncatedWrongValueForField) + tk.MustGetErrCode("insert into t values('1e')", errno.ErrTruncatedWrongValueForField) + tk.MustGetErrCode("insert into t values('1.2A')", errno.ErrTruncatedWrongValueForField) + tk.MustGetErrCode("insert into t values('1.2.3.4.5')", errno.ErrTruncatedWrongValueForField) + tk.MustGetErrCode("insert into t values('1.2.')", errno.ErrTruncatedWrongValueForField) + tk.MustGetErrCode("insert into t values('1,999.00')", errno.ErrTruncatedWrongValueForField) + tk.MustExec("insert into t values('12e-3')") + tk.MustQuery("show warnings;").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect DECIMAL value: '0.012'")) + tk.MustQuery("select id from t").Check(testkit.Rows("0")) + tk.MustExec("drop table if exists t") +} + +func (s *testSuite13) TestIssue17745(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test`) + tk.MustExec("drop table if exists tt1") + tk.MustExec("create table tt1 (c1 decimal(64))") + tk.MustGetErrCode("insert into tt1 values(89000000000000000000000000000000000000000000000000000000000000000000000000000000000000000)", errno.ErrWarnDataOutOfRange) + tk.MustGetErrCode("insert into tt1 values(89123456789012345678901234567890123456789012345678901234567890123456789012345678900000000)", errno.ErrWarnDataOutOfRange) + tk.MustExec("insert ignore into tt1 values(89123456789012345678901234567890123456789012345678901234567890123456789012345678900000000)") + tk.MustQuery("show warnings;").Check(testkit.Rows(`Warning 1690 DECIMAL value is out of range in '(64, 0)'`, `Warning 1292 Truncated incorrect DECIMAL value: '789012345678901234567890123456789012345678901234567890123456789012345678900000000'`)) + tk.MustQuery("select c1 from tt1").Check(testkit.Rows("9999999999999999999999999999999999999999999999999999999999999999")) + tk.MustGetErrCode("update tt1 set c1 = 89123456789012345678901234567890123456789012345678901234567890123456789012345678900000000", errno.ErrWarnDataOutOfRange) + tk.MustExec("drop table if exists tt1") + tk.MustGetErrCode("insert into tt1 values(4556414e723532)", errno.ErrIllegalValueForType) + tk.MustQuery("select 888888888888888888888888888888888888888888888888888888888888888888888888888888888888").Check(testkit.Rows("99999999999999999999999999999999999999999999999999999999999999999")) + tk.MustQuery("show warnings;").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect DECIMAL value: '888888888888888888888888888888888888888888888888888888888888888888888888888888888'")) +} + +// TestInsertIssue29892 test the double type with auto_increment problem, just leverage the serial test suite. +func (s *testAutoRandomSuite) TestInsertIssue29892(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test`) + + tk.MustExec("set global tidb_txn_mode='optimistic';") + tk.MustExec("set global tidb_disable_txn_auto_retry=false;") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a double auto_increment key, b int)") + tk.MustExec("insert into t values (146576794, 1)") + + tk1 := testkit.NewTestKit(c, s.store) + tk1.MustExec(`use test`) + tk1.MustExec("begin") + tk1.MustExec("insert into t(b) select 1") + + tk2 := testkit.NewTestKit(c, s.store) + tk2.MustExec(`use test`) + tk2.MustExec("begin") + tk2.MustExec("insert into t values (146576795, 1)") + tk2.MustExec("insert into t values (146576796, 1)") + tk2.MustExec("commit") + + // since the origin auto-id (146576795) is cached in retryInfo, it will be fetched again to do the retry again, + // which will duplicate with what has been inserted in tk1. + _, err := tk1.Exec("commit") + c.Assert(err, NotNil) + c.Assert(strings.Contains(err.Error(), "Duplicate entry"), Equals, true) +} diff --git a/executor/join_test.go b/executor/join_test.go index fd42303ba1340..7026c2c87dfb3 100644 --- a/executor/join_test.go +++ b/executor/join_test.go @@ -656,6 +656,39 @@ func (s *testSuiteJoin1) TestUsing(c *C) { tk.MustQuery("select t1.t0, t2.t0 from t1 join t2 using(t0) having t1.t0 > 0").Check(testkit.Rows("1 1")) } +func (s *testSuiteWithData) TestUsingAndNaturalJoinSchema(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2, t3, t4") + tk.MustExec("create table t1 (c int, b int);") + tk.MustExec("create table t2 (a int, b int);") + tk.MustExec("create table t3 (b int, c int);") + tk.MustExec("create table t4 (y int, c int);") + + tk.MustExec("insert into t1 values (10,1);") + tk.MustExec("insert into t1 values (3 ,1);") + tk.MustExec("insert into t1 values (3 ,2);") + tk.MustExec("insert into t2 values (2, 1);") + tk.MustExec("insert into t3 values (1, 3);") + tk.MustExec("insert into t3 values (1,10);") + tk.MustExec("insert into t4 values (11,3);") + tk.MustExec("insert into t4 values (2, 3);") + + var input []string + var output []struct { + SQL string + Res []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + }) + tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) + } +} + func (s *testSuiteWithData) TestNaturalJoin(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/executor/main_test.go b/executor/main_test.go index dd7d11b3c8274..1e02955ee8e4c 100644 --- a/executor/main_test.go +++ b/executor/main_test.go @@ -12,22 +12,35 @@ // See the License for the specific language governing permissions and // limitations under the License. -package executor +package executor_test import ( + "fmt" "os" "testing" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/testkit/testmain" "github.com/pingcap/tidb/util/testbridge" "github.com/tikv/client-go/v2/tikv" "go.uber.org/goleak" ) +var testDataMap = make(testdata.BookKeeper) +var prepareMergeSuiteData testdata.TestData +var aggMergeSuiteData testdata.TestData + func TestMain(m *testing.M) { testbridge.WorkaroundGoCheckFlags() + testDataMap.LoadTestSuiteData("testdata", "prepare_suite") + testDataMap.LoadTestSuiteData("testdata", "agg_suite") + prepareMergeSuiteData = testDataMap["prepare_suite"] + aggMergeSuiteData = testDataMap["agg_suite"] + autoid.SetStep(5000) config.UpdateGlobal(func(conf *config.Config) { conf.Log.SlowThreshold = 30000 // 30s @@ -46,5 +59,19 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"), goleak.IgnoreTopFunction("github.com/pingcap/tidb/table/tables.mockRemoteService"), } - goleak.VerifyTestMain(m, opts...) + callback := func(i int) int { + testDataMap.GenerateOutputIfNeeded() + return i + } + + goleak.VerifyTestMain(testmain.WrapTestingM(m, callback), opts...) +} + +func fillData(tk *testkit.TestKit, table string) { + tk.MustExec("use test") + tk.MustExec(fmt.Sprintf("create table %s(id int not null default 1, name varchar(255), PRIMARY KEY(id));", table)) + + // insert data + tk.MustExec(fmt.Sprintf("insert INTO %s VALUES (1, \"hello\");", table)) + tk.MustExec(fmt.Sprintf("insert into %s values (2, \"hello\");", table)) } diff --git a/executor/memtable_reader.go b/executor/memtable_reader.go index 70d091de84849..1f014e53d26ae 100644 --- a/executor/memtable_reader.go +++ b/executor/memtable_reader.go @@ -15,6 +15,7 @@ package executor import ( + "bytes" "container/heap" "context" "encoding/json" @@ -39,18 +40,22 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/set" + "github.com/tikv/client-go/v2/tikv" "go.uber.org/zap" "google.golang.org/grpc" "google.golang.org/grpc/credentials" ) const clusterLogBatchSize = 256 +const hotRegionsHistoryBatchSize = 256 type dummyCloser struct{} @@ -696,3 +701,299 @@ func (e *clusterLogRetriever) close() error { func (e *clusterLogRetriever) getRuntimeStats() execdetails.RuntimeStats { return nil } + +type hotRegionsResult struct { + addr string + messages *HistoryHotRegions + err error +} + +type hotRegionsResponseHeap []hotRegionsResult + +func (h hotRegionsResponseHeap) Len() int { + return len(h) +} + +func (h hotRegionsResponseHeap) Less(i, j int) bool { + lhs, rhs := h[i].messages.HistoryHotRegion[0], h[j].messages.HistoryHotRegion[0] + if lhs.UpdateTime != rhs.UpdateTime { + return lhs.UpdateTime < rhs.UpdateTime + } + return lhs.HotDegree < rhs.HotDegree +} + +func (h hotRegionsResponseHeap) Swap(i, j int) { + h[i], h[j] = h[j], h[i] +} + +func (h *hotRegionsResponseHeap) Push(x interface{}) { + *h = append(*h, x.(hotRegionsResult)) +} + +func (h *hotRegionsResponseHeap) Pop() interface{} { + old := *h + n := len(old) + x := old[n-1] + *h = old[0 : n-1] + return x +} + +type hotRegionsHistoryRetriver struct { + dummyCloser + isDrained bool + retrieving bool + heap *hotRegionsResponseHeap + extractor *plannercore.HotRegionsHistoryTableExtractor +} + +// HistoryHotRegionsRequest wrap conditions push down to PD. +type HistoryHotRegionsRequest struct { + StartTime int64 `json:"start_time,omitempty"` + EndTime int64 `json:"end_time,omitempty"` + RegionIDs []uint64 `json:"region_ids,omitempty"` + StoreIDs []uint64 `json:"store_ids,omitempty"` + PeerIDs []uint64 `json:"peer_ids,omitempty"` + IsLearners []bool `json:"is_learners,omitempty"` + IsLeaders []bool `json:"is_leaders,omitempty"` + HotRegionTypes []string `json:"hot_region_type,omitempty"` +} + +// HistoryHotRegions records filtered hot regions stored in each PD. +// it's the response of PD. +type HistoryHotRegions struct { + HistoryHotRegion []*HistoryHotRegion `json:"history_hot_region"` +} + +// HistoryHotRegion records each hot region's statistics. +// it's the response of PD. +type HistoryHotRegion struct { + UpdateTime int64 `json:"update_time,omitempty"` + RegionID uint64 `json:"region_id,omitempty"` + StoreID uint64 `json:"store_id,omitempty"` + PeerID uint64 `json:"peer_id,omitempty"` + IsLearner bool `json:"is_learner,omitempty"` + IsLeader bool `json:"is_leader,omitempty"` + HotRegionType string `json:"hot_region_type,omitempty"` + HotDegree int64 `json:"hot_degree,omitempty"` + FlowBytes float64 `json:"flow_bytes,omitempty"` + KeyRate float64 `json:"key_rate,omitempty"` + QueryRate float64 `json:"query_rate,omitempty"` + StartKey []byte `json:"start_key,omitempty"` + EndKey []byte `json:"end_key,omitempty"` +} + +func (e *hotRegionsHistoryRetriver) initialize(ctx context.Context, sctx sessionctx.Context) ([]chan hotRegionsResult, error) { + if !hasPriv(sctx, mysql.ProcessPriv) { + return nil, plannercore.ErrSpecificAccessDenied.GenWithStackByArgs("PROCESS") + } + pdServers, err := infoschema.GetPDServerInfo(sctx) + if err != nil { + return nil, err + } + + // To avoid search hot regions interface overload, the user should specify the time range in normally SQL. + if e.extractor.StartTime == 0 { + return nil, errors.New("denied to scan hot regions, please specified the start time, such as `update_time > '2020-01-01 00:00:00'`") + } + if e.extractor.EndTime == 0 { + return nil, errors.New("denied to scan hot regions, please specified the end time, such as `update_time < '2020-01-01 00:00:00'`") + } + + historyHotRegionsRequest := &HistoryHotRegionsRequest{ + StartTime: e.extractor.StartTime, + EndTime: e.extractor.EndTime, + RegionIDs: e.extractor.RegionIDs, + StoreIDs: e.extractor.StoreIDs, + PeerIDs: e.extractor.PeerIDs, + IsLearners: e.extractor.IsLearners, + IsLeaders: e.extractor.IsLeaders, + } + + return e.startRetrieving(ctx, sctx, pdServers, historyHotRegionsRequest) +} + +func (e *hotRegionsHistoryRetriver) startRetrieving( + ctx context.Context, + sctx sessionctx.Context, + pdServers []infoschema.ServerInfo, + req *HistoryHotRegionsRequest, +) ([]chan hotRegionsResult, error) { + + var results []chan hotRegionsResult + for _, srv := range pdServers { + for typ := range e.extractor.HotRegionTypes { + req.HotRegionTypes = []string{typ} + jsonBody, err := json.Marshal(req) + if err != nil { + return nil, err + } + body := bytes.NewBuffer(jsonBody) + ch := make(chan hotRegionsResult) + results = append(results, ch) + go func(ch chan hotRegionsResult, address string, body *bytes.Buffer) { + util.WithRecovery(func() { + defer close(ch) + url := fmt.Sprintf("%s://%s%s", util.InternalHTTPSchema(), address, pdapi.HotHistory) + req, err := http.NewRequest(http.MethodGet, url, body) + if err != nil { + ch <- hotRegionsResult{err: errors.Trace(err)} + return + } + req.Header.Add("PD-Allow-follower-handle", "true") + resp, err := util.InternalHTTPClient().Do(req) + if err != nil { + ch <- hotRegionsResult{err: errors.Trace(err)} + return + } + defer func() { + terror.Log(resp.Body.Close()) + }() + if resp.StatusCode != http.StatusOK { + ch <- hotRegionsResult{err: errors.Errorf("request %s failed: %s", url, resp.Status)} + return + } + var historyHotRegions HistoryHotRegions + if err = json.NewDecoder(resp.Body).Decode(&historyHotRegions); err != nil { + ch <- hotRegionsResult{err: errors.Trace(err)} + return + } + ch <- hotRegionsResult{addr: address, messages: &historyHotRegions} + }, nil) + }(ch, srv.StatusAddr, body) + } + } + return results, nil +} + +func (e *hotRegionsHistoryRetriver) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) { + if e.extractor.SkipRequest || e.isDrained { + return nil, nil + } + + if !e.retrieving { + e.retrieving = true + results, err := e.initialize(ctx, sctx) + if err != nil { + e.isDrained = true + return nil, err + } + // Initialize the heap + e.heap = &hotRegionsResponseHeap{} + for _, ch := range results { + result := <-ch + if result.err != nil || len(result.messages.HistoryHotRegion) == 0 { + if result.err != nil { + sctx.GetSessionVars().StmtCtx.AppendWarning(result.err) + } + continue + } + *e.heap = append(*e.heap, result) + } + heap.Init(e.heap) + } + // Merge the results + var finalRows [][]types.Datum + allSchemas := sctx.GetInfoSchema().(infoschema.InfoSchema).AllSchemas() + tz := sctx.GetSessionVars().Location() + tikvStore, ok := sctx.GetStore().(helper.Storage) + if !ok { + return nil, errors.New("Information about hot region can be gotten only when the storage is TiKV") + } + tikvHelper := &helper.Helper{ + Store: tikvStore, + RegionCache: tikvStore.GetRegionCache(), + } + for e.heap.Len() > 0 && len(finalRows) < hotRegionsHistoryBatchSize { + minTimeItem := heap.Pop(e.heap).(hotRegionsResult) + row, err := e.getHotRegionRowWithSchemaInfo(minTimeItem.messages.HistoryHotRegion[0], tikvHelper, allSchemas, tz) + if err != nil { + return nil, err + } + if row != nil { + finalRows = append(finalRows, row) + } + minTimeItem.messages.HistoryHotRegion = minTimeItem.messages.HistoryHotRegion[1:] + // Fetch next message item + if len(minTimeItem.messages.HistoryHotRegion) != 0 { + heap.Push(e.heap, minTimeItem) + } + } + // All streams are drained + e.isDrained = e.heap.Len() == 0 + return finalRows, nil +} + +func (e *hotRegionsHistoryRetriver) getHotRegionRowWithSchemaInfo( + hisHotRegion *HistoryHotRegion, + tikvHelper *helper.Helper, + allSchemas []*model.DBInfo, + tz *time.Location, +) ([]types.Datum, error) { + _, startKey, _ := codec.DecodeBytes(hisHotRegion.StartKey, []byte{}) + _, endKey, _ := codec.DecodeBytes(hisHotRegion.EndKey, []byte{}) + region := &tikv.KeyLocation{StartKey: startKey, EndKey: endKey} + hotRange, err := helper.NewRegionFrameRange(region) + if err != nil { + return nil, err + } + + f := tikvHelper.FindTableIndexOfRegion(allSchemas, hotRange) + // Ignore row without corresponding schema f. + if f == nil { + return nil, nil + } + row := make([]types.Datum, len(infoschema.TableTiDBHotRegionsHistoryCols)) + updateTimestamp := time.Unix(hisHotRegion.UpdateTime/1000, (hisHotRegion.UpdateTime%1000)*int64(time.Millisecond)) + + if updateTimestamp.Location() != tz { + updateTimestamp.In(tz) + } + updateTime := types.NewTime(types.FromGoTime(updateTimestamp), mysql.TypeTimestamp, types.MinFsp) + row[0].SetMysqlTime(updateTime) + row[1].SetString(strings.ToUpper(f.DBName), mysql.DefaultCollationName) + row[2].SetString(strings.ToUpper(f.TableName), mysql.DefaultCollationName) + row[3].SetInt64(f.TableID) + if f.IndexName != "" { + row[4].SetString(strings.ToUpper(f.IndexName), mysql.DefaultCollationName) + row[5].SetInt64(f.IndexID) + } else { + row[4].SetNull() + row[5].SetNull() + } + row[6].SetInt64(int64(hisHotRegion.RegionID)) + row[7].SetInt64(int64(hisHotRegion.StoreID)) + row[8].SetInt64(int64(hisHotRegion.PeerID)) + if hisHotRegion.IsLearner { + row[9].SetInt64(1) + } else { + row[9].SetInt64(0) + } + if hisHotRegion.IsLeader { + row[10].SetInt64(1) + } else { + row[10].SetInt64(0) + } + + row[11].SetString(strings.ToUpper(hisHotRegion.HotRegionType), mysql.DefaultCollationName) + if hisHotRegion.HotDegree != 0 { + row[12].SetInt64(hisHotRegion.HotDegree) + } else { + row[12].SetNull() + } + if hisHotRegion.FlowBytes != 0 { + row[13].SetFloat64(hisHotRegion.FlowBytes) + } else { + row[13].SetNull() + } + if hisHotRegion.KeyRate != 0 { + row[14].SetFloat64(hisHotRegion.KeyRate) + } else { + row[14].SetNull() + } + if hisHotRegion.QueryRate != 0 { + row[15].SetFloat64(hisHotRegion.QueryRate) + } else { + row[15].SetNull() + } + return row, nil +} diff --git a/executor/memtable_reader_test.go b/executor/memtable_reader_test.go index 817741c2edd21..9fa51faaa3737 100644 --- a/executor/memtable_reader_test.go +++ b/executor/memtable_reader_test.go @@ -16,9 +16,13 @@ package executor_test import ( "context" + "crypto/tls" + "encoding/json" "fmt" + "io" "log" "net" + "net/http" "net/http/httptest" "os" "path/filepath" @@ -33,8 +37,11 @@ import ( "github.com/pingcap/kvproto/pkg/diagnosticspb" "github.com/pingcap/sysutil" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/testkit" pmodel "github.com/prometheus/common/model" @@ -956,3 +963,441 @@ func (s *testMemTableReaderSuite) TestTiDBClusterLogError(c *C) { c.Assert(err.Error(), Equals, "denied to scan full logs (use `SELECT * FROM cluster_log WHERE message LIKE '%'` explicitly if intentionally)") c.Assert(rs.Close(), IsNil) } + +type mockStoreWithMultiPD struct { + helper.Storage + hosts []string +} + +var hotRegionsResponses = make(map[string]*executor.HistoryHotRegions, 3) + +func (s *mockStoreWithMultiPD) EtcdAddrs() ([]string, error) { return s.hosts, nil } +func (s *mockStoreWithMultiPD) TLSConfig() *tls.Config { panic("not implemented") } +func (s *mockStoreWithMultiPD) StartGCWorker() error { panic("not implemented") } +func (s *mockStoreWithMultiPD) Name() string { return "mockStore" } +func (s *mockStoreWithMultiPD) Describe() string { return "" } + +var _ = SerialSuites(&testHotRegionsHistoryTableSuite{testInfoschemaTableSuiteBase: &testInfoschemaTableSuiteBase{}}) + +type testHotRegionsHistoryTableSuite struct { + *testInfoschemaTableSuiteBase + httpServers []*httptest.Server + startTime time.Time +} + +func (s *testHotRegionsHistoryTableSuite) SetUpSuite(c *C) { + s.testInfoschemaTableSuiteBase.SetUpSuite(c) + store := &mockStoreWithMultiPD{ + s.store.(helper.Storage), + make([]string, 3), + } + // start 3 PD server with hotRegionsServer and store them in s.store + for i := 0; i < 3; i++ { + httpServer, mockAddr := s.setUpMockPDHTTPServer(c) + c.Assert(httpServer, NotNil) + s.httpServers = append(s.httpServers, httpServer) + store.hosts[i] = mockAddr + } + s.store = store + s.startTime = time.Now() +} + +func writeJSONError(w http.ResponseWriter, code int, prefix string, err error) { + type errorResponse struct { + Error string `json:"error"` + } + w.WriteHeader(code) + if err != nil { + prefix += ": " + err.Error() + } + _ = json.NewEncoder(w).Encode(errorResponse{Error: prefix}) +} + +func hisHotRegionsHandler(w http.ResponseWriter, r *http.Request) { + w.Header().Set("Access-Control-Allow-Origin", "*") + data, err := io.ReadAll(r.Body) + if err != nil { + writeJSONError(w, http.StatusInternalServerError, "unable to read req", err) + return + } + r.Body.Close() + req := &executor.HistoryHotRegionsRequest{} + err = json.Unmarshal(data, req) + if err != nil { + writeJSONError(w, http.StatusInternalServerError, "unable to serialize req", err) + return + } + resp := &executor.HistoryHotRegions{} + for _, typ := range req.HotRegionTypes { + resp.HistoryHotRegion = append(resp.HistoryHotRegion, hotRegionsResponses[typ+r.Host].HistoryHotRegion...) + } + w.WriteHeader(http.StatusOK) + jsonResp, err := json.Marshal(resp) + if err != nil { + writeJSONError(w, http.StatusInternalServerError, "unable to marshal resp", err) + return + } + w.Write(jsonResp) +} + +func (s *testHotRegionsHistoryTableSuite) setUpMockPDHTTPServer(c *C) (*httptest.Server, string) { + // mock PD http server + router := mux.NewRouter() + server := httptest.NewServer(router) + mockAddr := strings.TrimPrefix(server.URL, "http://") + // mock PD API + router.Handle(pdapi.Status, fn.Wrap(func() (interface{}, error) { + return struct { + Version string `json:"version"` + GitHash string `json:"git_hash"` + StartTimestamp int64 `json:"start_timestamp"` + }{ + Version: "4.0.0-alpha", + GitHash: "mock-pd-githash", + StartTimestamp: s.startTime.Unix(), + }, nil + })) + // mock hisory hot regions response + router.HandleFunc(pdapi.HotHistory, hisHotRegionsHandler) + return server, mockAddr +} + +func (s *testHotRegionsHistoryTableSuite) TearDownSuite(c *C) { + for _, server := range s.httpServers { + server.Close() + } + s.testInfoschemaTableSuiteBase.TearDownSuite(c) +} + +func (s *testHotRegionsHistoryTableSuite) TestTiDBHotRegionsHistory(c *C) { + var unixTimeMs = func(s string) int64 { + t, err := time.ParseInLocation("2006-01-02 15:04:05", s, time.Local) + c.Assert(err, IsNil) + return t.UnixNano() / int64(time.Millisecond) + } + fullHotRegions := [][]string{ + // mysql table_id = 11, record_id = 1, table_name = TABLES_PRIV + {"2019-10-10 10:10:11", "MYSQL", "TABLES_PRIV", "11", "", "", "1", "1", "11111", "0", "1", "READ", "99", "99", "99", "99"}, + {"2019-10-10 10:10:12", "MYSQL", "TABLES_PRIV", "11", "", "", "2", "2", "22222", "0", "0", "WRITE", "99", "99", "99", "99"}, + // mysql table_id = 21, record_id = 1, table_name = STATS_META + {"2019-10-10 10:10:13", "MYSQL", "STATS_META", "21", "", "", "3", "3", "33333", "0", "1", "READ", "99", "99", "99", "99"}, + {"2019-10-10 10:10:14", "MYSQL", "STATS_META", "21", "", "", "4", "4", "44444", "0", "0", "WRITE", "99", "99", "99", "99"}, + // table_id = 131, record_id=1, deleted schema + {"2019-10-10 10:10:15", "UNKONW", "UNKONW", "131", "UNKONW", "", "5", "5", "55555", "0", "1", "READ", "99", "99", "99", "99"}, + {"2019-10-10 10:10:16", "UNKONW", "UNKONW", "131", "UNKONW", "", "6", "6", "66666", "0", "0", "WRITE", "99", "99", "99", "99"}, + // mysql table_id = 11, index_id = 1, index_value = 1, table_name = TABLES_PRIV, index_name = PRIMARY + {"2019-10-10 10:10:17", "MYSQL", "TABLES_PRIV", "11", "PRIMARY", "1", "1", "1", "11111", "0", "1", "READ", "99", "99", "99", "99"}, + {"2019-10-10 10:10:18", "MYSQL", "TABLES_PRIV", "11", "PRIMARY", "1", "2", "2", "22222", "0", "0", "WRITE", "99", "99", "99", "99"}, + // mysql table_id = 21 ,index_id = 1, index_value = 1, table_name = STATS_META, index_name = IDX_VER + {"2019-10-10 10:10:19", "MYSQL", "STATS_META", "21", "IDX_VER", "1", "3", "3", "33333", "0", "1", "READ", "99", "99", "99", "99"}, + {"2019-10-10 10:10:20", "MYSQL", "STATS_META", "21", "IDX_VER", "1", "4", "4", "44444", "0", "0", "WRITE", "99", "99", "99", "99"}, + // mysql table_id = 21 ,index_id = 2, index_value = 1, table_name = STATS_META, index_name = TBL + {"2019-10-10 10:10:21", "MYSQL", "STATS_META", "21", "TBL", "2", "5", "5", "55555", "0", "1", "READ", "99", "99", "99", "99"}, + {"2019-10-10 10:10:22", "MYSQL", "STATS_META", "21", "TBL", "2", "6", "6", "66666", "0", "0", "WRITE", "99", "99", "99", "99"}, + // table_id = 131, index_id = 1, index_value = 1, deleted schema + {"2019-10-10 10:10:23", "UNKONW", "UNKONW", "131", "UNKONW", "1", "5", "5", "55555", "0", "1", "READ", "99", "99", "99", "99"}, + {"2019-10-10 10:10:24", "UNKONW", "UNKONW", "131", "UNKONW", "1", "6", "6", "66666", "0", "0", "WRITE", "99", "99", "99", "99"}, + } + + pdResps := []map[string]*executor.HistoryHotRegions{ + { + core.HotRegionTypeRead: { + HistoryHotRegion: []*executor.HistoryHotRegion{ + // mysql table_id = 11, record_id = 1, table_name = TABLES_PRIV + {UpdateTime: unixTimeMs("2019-10-10 10:10:11"), RegionID: 1, StoreID: 1, PeerID: 11111, IsLearner: false, IsLeader: true, HotRegionType: "READ", HotDegree: 99, FlowBytes: 99, KeyRate: 99, QueryRate: 99, + StartKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0xb, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}, + EndKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0xb, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}}, + // mysql table_id = 21, record_id = 1, table_name = STATS_META + {UpdateTime: unixTimeMs("2019-10-10 10:10:13"), RegionID: 3, StoreID: 3, PeerID: 33333, IsLearner: false, IsLeader: true, HotRegionType: "READ", HotDegree: 99, FlowBytes: 99, KeyRate: 99, QueryRate: 99, + StartKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x15, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}, + EndKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x15, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}}, + }, + }, + core.HotRegionTypeWrite: { + HistoryHotRegion: []*executor.HistoryHotRegion{ + // mysql table_id = 11, record_id = 1, table_name = TABLES_PRIV + {UpdateTime: unixTimeMs("2019-10-10 10:10:12"), RegionID: 2, StoreID: 2, PeerID: 22222, IsLearner: false, IsLeader: false, HotRegionType: "WRITE", HotDegree: 99, FlowBytes: 99, KeyRate: 99, QueryRate: 99, + StartKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0xb, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}, + EndKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0xb, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}}, + // mysql table_id = 21, record_id = 1, table_name = STATS_META + {UpdateTime: unixTimeMs("2019-10-10 10:10:14"), RegionID: 4, StoreID: 4, PeerID: 44444, IsLearner: false, IsLeader: false, HotRegionType: "WRITE", HotDegree: 99, FlowBytes: 99, KeyRate: 99, QueryRate: 99, + StartKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x15, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}, + EndKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x15, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}}, + }, + }, + }, + { + core.HotRegionTypeRead: { + HistoryHotRegion: []*executor.HistoryHotRegion{ + // table_id = 131, record_id=1, deleted schema + {UpdateTime: unixTimeMs("2019-10-10 10:10:15"), RegionID: 5, StoreID: 5, PeerID: 55555, IsLearner: false, IsLeader: true, HotRegionType: "READ", HotDegree: 99, FlowBytes: 99, KeyRate: 99, QueryRate: 99, + StartKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x83, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}, + EndKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x83, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}}, + // mysql table_id = 11, index_id = 1, index_value = 1, table_name = TABLES_PRIV, index_name = PRIMARY + {UpdateTime: unixTimeMs("2019-10-10 10:10:17"), RegionID: 1, StoreID: 1, PeerID: 11111, IsLearner: false, IsLeader: true, HotRegionType: "READ", HotDegree: 99, FlowBytes: 99, KeyRate: 99, QueryRate: 99, + StartKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0xb, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}, + EndKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0xb, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}}, + }, + }, + core.HotRegionTypeWrite: { + HistoryHotRegion: []*executor.HistoryHotRegion{ + // table_id = 131, record_id=1, deleted schema + {UpdateTime: unixTimeMs("2019-10-10 10:10:16"), RegionID: 6, StoreID: 6, PeerID: 66666, IsLearner: false, IsLeader: false, HotRegionType: "WRITE", HotDegree: 99, FlowBytes: 99, KeyRate: 99, QueryRate: 99, + StartKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x83, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}, + EndKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x83, 0x5f, 0x72, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}}, + // mysql table_id = 11, index_id = 1, index_value = 1, table_name = TABLES_PRIV, index_name = PRIMARY + {UpdateTime: unixTimeMs("2019-10-10 10:10:18"), RegionID: 2, StoreID: 2, PeerID: 22222, IsLearner: false, IsLeader: false, HotRegionType: "WRITE", HotDegree: 99, FlowBytes: 99, KeyRate: 99, QueryRate: 99, + StartKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0xb, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}, + EndKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0xb, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}}, + }, + }, + }, + { + core.HotRegionTypeRead: { + HistoryHotRegion: []*executor.HistoryHotRegion{ + // mysql table_id = 21 ,index_id = 1, index_value = 1, table_name = STATS_META, index_name = IDX_VER + {UpdateTime: unixTimeMs("2019-10-10 10:10:19"), RegionID: 3, StoreID: 3, PeerID: 33333, IsLearner: false, IsLeader: true, HotRegionType: "READ", HotDegree: 99, FlowBytes: 99, KeyRate: 99, QueryRate: 99, + StartKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x15, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}, + EndKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x15, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}}, + // mysql table_id = 21 ,index_id = 2, index_value = 1, table_name = STATS_META, index_name = TBL + {UpdateTime: unixTimeMs("2019-10-10 10:10:21"), RegionID: 5, StoreID: 5, PeerID: 55555, IsLearner: false, IsLeader: true, HotRegionType: "READ", HotDegree: 99, FlowBytes: 99, KeyRate: 99, QueryRate: 99, + StartKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x15, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x2, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}, + EndKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x15, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x2, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}}, + // table_id = 131, index_id = 1, index_value = 1, deleted schema + {UpdateTime: unixTimeMs("2019-10-10 10:10:23"), RegionID: 5, StoreID: 5, PeerID: 55555, IsLeader: true, HotRegionType: "READ", HotDegree: 99, FlowBytes: 99, KeyRate: 99, QueryRate: 99, + StartKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x83, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}, + EndKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x83, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}}, + }, + }, + core.HotRegionTypeWrite: { + HistoryHotRegion: []*executor.HistoryHotRegion{ + // mysql table_id = 21 ,index_id = 1, index_value = 1, table_name = STATS_META, index_name = IDX_VER + {UpdateTime: unixTimeMs("2019-10-10 10:10:20"), RegionID: 4, StoreID: 4, PeerID: 44444, IsLearner: false, IsLeader: false, HotRegionType: "WRITE", HotDegree: 99, FlowBytes: 99, KeyRate: 99, QueryRate: 99, + StartKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x15, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}, + EndKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x15, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}}, + // mysql table_id = 21 ,index_id = 2, index_value = 1, table_name = STATS_META, index_name = TBL + {UpdateTime: unixTimeMs("2019-10-10 10:10:22"), RegionID: 6, StoreID: 6, PeerID: 66666, IsLearner: false, IsLeader: false, HotRegionType: "WRITE", HotDegree: 99, FlowBytes: 99, KeyRate: 99, QueryRate: 99, + StartKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x15, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x2, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}, + EndKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x15, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x2, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}}, + // table_id = 131, index_id = 1, index_value = 1, deleted schema + {UpdateTime: unixTimeMs("2019-10-10 10:10:24"), RegionID: 6, StoreID: 6, PeerID: 66666, IsLearner: false, IsLeader: false, HotRegionType: "WRITE", HotDegree: 99, FlowBytes: 99, KeyRate: 99, QueryRate: 99, + StartKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x83, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}, + EndKey: []byte{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0xff, 0x83, 0x5f, 0x69, 0x80, 0x0, 0x0, 0x0, 0x0, 0xff, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0, 0x0, 0x0, 0xfa}}, + }, + }, + }, + } + + var cases = []struct { + conditions []string + reqCount int32 + expected [][]string + }{ + { + conditions: []string{ + "update_time>='2019-10-10 10:10:10'", + "update_time<='2019-10-11 10:10:10'", + }, // time filtered by PD, assume response suit time range, and ignore deleted schemas + expected: [][]string{ + fullHotRegions[0], fullHotRegions[1], fullHotRegions[2], + fullHotRegions[3], + fullHotRegions[6], fullHotRegions[7], fullHotRegions[8], + fullHotRegions[9], fullHotRegions[10], fullHotRegions[11], + }, + }, + { + conditions: []string{ + "update_time>=TIMESTAMP('2019-10-10 10:10:10')", + "update_time<=TIMESTAMP('2019-10-11 10:10:10')", + }, // test support of timestamp + expected: [][]string{ + fullHotRegions[0], fullHotRegions[1], fullHotRegions[2], + fullHotRegions[3], + fullHotRegions[6], fullHotRegions[7], fullHotRegions[8], + fullHotRegions[9], fullHotRegions[10], fullHotRegions[11], + }, + }, + { + conditions: []string{ + "update_time>='2019-10-10 10:10:10'", + "update_time<='2019-10-11 10:10:10'", + "table_id=11", + }, + expected: [][]string{ + fullHotRegions[0], fullHotRegions[1], fullHotRegions[6], fullHotRegions[7], + }, + }, + { + conditions: []string{ + "update_time>='2019-10-10 10:10:10'", + "update_time<='2019-10-11 10:10:10'", + "table_name='TABLES_PRIV'", + }, + expected: [][]string{ + fullHotRegions[0], fullHotRegions[1], fullHotRegions[6], fullHotRegions[7], + }, + }, + { + conditions: []string{ + "update_time>='2019-10-10 10:10:10'", + "update_time<='2019-10-11 10:10:10'", + "table_id=21", + "index_id=1", + }, + expected: [][]string{ + fullHotRegions[8], fullHotRegions[9], + }, + }, + { + conditions: []string{ + "update_time>='2019-10-10 10:10:10'", + "update_time<='2019-10-11 10:10:10'", + "table_id=21", + "index_id=1", + "table_name='TABLES_PRIV'", + }, // table_id != table_name -> nil + expected: [][]string{}, + }, + { + conditions: []string{ + "update_time>='2019-10-10 10:10:10'", + "update_time<='2019-10-11 10:10:10'", + "table_id=21", + "index_id=1", + "table_name='STATS_META'", + }, // table_id = table_name + expected: [][]string{ + fullHotRegions[8], fullHotRegions[9], + }, + }, + { + conditions: []string{ + "update_time>='2019-10-10 10:10:10'", + "update_time<='2019-10-11 10:10:10'", + "table_id=21", + "index_id=1", + "index_name='UNKONW'", + }, // index_id != index_name -> nil + expected: [][]string{}, + }, + { + conditions: []string{ + "update_time>='2019-10-10 10:10:10'", + "update_time<='2019-10-11 10:10:10'", + "table_id=21", + "index_id=1", + "index_name='IDX_VER'", + }, // index_id = index_name + expected: [][]string{ + fullHotRegions[8], fullHotRegions[9], + }, + }, + { + conditions: []string{ + "update_time>='2019-10-10 10:10:10'", + "update_time<='2019-10-11 10:10:10'", + "index_id=1", + "index_name='IDX_VER'", + "table_id>=21", // unpushed down predicates 21>=21 + }, + expected: [][]string{ + fullHotRegions[8], fullHotRegions[9], + }, + }, + { + conditions: []string{ + "update_time>='2019-10-10 10:10:10'", + "update_time<='2019-10-11 10:10:10'", + "index_id=1", + "index_name='IDX_VER'", + "table_id>21", // unpushed down predicates + }, // 21!>21 -> nil + expected: [][]string{}, + }, + { + conditions: []string{ + "update_time>='2019-10-10 10:10:10'", + "update_time<='2019-10-11 10:10:10'", + "index_id=1", + "index_name='IDX_VER'", + "table_id>=21", // unpushed down predicates + "db_name='MYSQL'", + }, + expected: [][]string{ + fullHotRegions[8], fullHotRegions[9], + }, + }, + { + conditions: []string{ + "update_time>='2019-10-10 10:10:10'", + "update_time<='2019-10-11 10:10:10'", + "index_id=1", + "index_name='IDX_VER'", + "table_id>=21", // unpushed down predicates + "db_name='MYSQL'", + "peer_id>=33334", + }, + expected: [][]string{ + fullHotRegions[9], + }, + }, + { + conditions: []string{ + "update_time>='2019-10-10 10:10:10'", + "update_time<='2019-10-11 10:10:10'", + "index_id=1", + "index_name='IDX_VER'", + "table_id>=21", // unpushed down predicates + "db_name='UNKNOW'", + }, + expected: [][]string{}, + }, + } + + // mock http resp + store := s.store.(*mockStoreWithMultiPD) + for i, resp := range pdResps { + for k, v := range resp { + hotRegionsResponses[k+store.hosts[i]] = v + } + } + tk := testkit.NewTestKit(c, s.store) + for _, cas := range cases { + sql := "select * from information_schema.tidb_hot_regions_history" + if len(cas.conditions) > 0 { + sql = fmt.Sprintf("%s where %s", sql, strings.Join(cas.conditions, " and ")) + } + result := tk.MustQuery(sql) + warnings := tk.Se.GetSessionVars().StmtCtx.GetWarnings() + c.Assert(len(warnings), Equals, 0, Commentf("unexpected warnigns: %+v, sql: %s", warnings, sql)) + var expected []string + for _, row := range cas.expected { + expectedRow := row + expected = append(expected, strings.Join(expectedRow, " ")) + } + result.Check(testkit.Rows(expected...)) + } +} + +func (s *testHotRegionsHistoryTableSuite) TestTiDBHotRegionsHistoryError(c *C) { + tk := testkit.NewTestKit(c, s.store) + fpName := "github.com/pingcap/tidb/executor/mockTiDBHotRegionsHistory" + c.Assert(failpoint.Enable(fpName, `return("")`), IsNil) + defer func() { c.Assert(failpoint.Disable(fpName), IsNil) }() + + // Test without start time error + rs, err := tk.Exec("select * from information_schema.tidb_hot_regions_history") + c.Assert(err, IsNil) + _, err = session.ResultSetToStringSlice(context.Background(), tk.Se, rs) + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "denied to scan hot regions, please specified the start time, such as `update_time > '2020-01-01 00:00:00'`") + c.Assert(rs.Close(), IsNil) + + // Test without end time error. + rs, err = tk.Exec("select * from information_schema.tidb_hot_regions_history where update_time>='2019/08/26 06:18:13.011'") + c.Assert(err, IsNil) + _, err = session.ResultSetToStringSlice(context.Background(), tk.Se, rs) + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "denied to scan hot regions, please specified the end time, such as `update_time < '2020-01-01 00:00:00'`") + c.Assert(rs.Close(), IsNil) +} diff --git a/executor/merge_join_test.go b/executor/merge_join_test.go index 9cf56fe1716c8..4d616513963e4 100644 --- a/executor/merge_join_test.go +++ b/executor/merge_join_test.go @@ -848,7 +848,6 @@ func (s *testSuiteJoin3) TestVectorizedShuffleMergeJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@session.tidb_merge_join_concurrency = 4;") tk.MustExec("use test") - tk.MustExec("use test") existTableMap := make(map[string]struct{}) runTest := func(ts1, ts2 []int) { getTable := func(prefix string, ts []int) string { diff --git a/executor/point_get.go b/executor/point_get.go index 489bbf9bb8085..45f3fa76e263f 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -190,7 +190,7 @@ func (e *PointGetExecutor) Open(context.Context) error { panic("point get replica option fail") } }) - setResourceGroupTagForTxn(e.ctx.GetSessionVars().StmtCtx, e.snapshot) + setResourceGroupTaggerForTxn(e.ctx.GetSessionVars().StmtCtx, e.snapshot) return nil } diff --git a/executor/prepared_serial_test.go b/executor/prepared_serial_test.go new file mode 100644 index 0000000000000..ae6f174bbd8c9 --- /dev/null +++ b/executor/prepared_serial_test.go @@ -0,0 +1,1184 @@ +// 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 ( + "fmt" + "strconv" + "strings" + "testing" + + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser/auth" + "github.com/pingcap/tidb/parser/model" + plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/israce" + "github.com/stretchr/testify/require" +) + +func TestIssue28064(t *testing.T) { + store, dom, err := newStoreWithBootstrap() + require.NoError(t, err) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + tk := testkit.NewTestKit(t, store) + defer func() { + dom.Close() + require.NoError(t, store.Close()) + }() + tk.MustExec("use test") + tk.MustExec("drop table if exists t28064") + tk.MustExec("CREATE TABLE `t28064` (" + + "`a` decimal(10,0) DEFAULT NULL," + + "`b` decimal(10,0) DEFAULT NULL," + + "`c` decimal(10,0) DEFAULT NULL," + + "`d` decimal(10,0) DEFAULT NULL," + + "KEY `iabc` (`a`,`b`,`c`));") + tk.MustExec("set @a='123', @b='234', @c='345';") + tk.MustExec("set @@tidb_enable_collect_execution_info=0;") + tk.MustExec("prepare stmt1 from 'select * from t28064 use index (iabc) where a = ? and b = ? and c = ?';") + + tk.MustExec("execute stmt1 using @a, @b, @c;") + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) + rows.Check(testkit.Rows("Selection_8 0.00 root eq(test.t28064.a, 123), eq(test.t28064.b, 234), eq(test.t28064.c, 345)", + "└─IndexLookUp_7 0.00 root ", + " ├─IndexRangeScan_5(Build) 0.00 cop[tikv] table:t28064, index:iabc(a, b, c) range:[123 234 345,123 234 345], keep order:false, stats:pseudo", + " └─TableRowIDScan_6(Probe) 0.00 cop[tikv] table:t28064 keep order:false, stats:pseudo")) + + tk.MustExec("execute stmt1 using @a, @b, @c;") + rows = tk.MustQuery("select @@last_plan_from_cache") + rows.Check(testkit.Rows("1")) + + tk.MustExec("execute stmt1 using @a, @b, @c;") + rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) + rows.Check(testkit.Rows("Selection_8 0.00 root eq(test.t28064.a, 123), eq(test.t28064.b, 234), eq(test.t28064.c, 345)", + "└─IndexLookUp_7 0.00 root ", + " ├─IndexRangeScan_5(Build) 0.00 cop[tikv] table:t28064, index:iabc(a, b, c) range:[123 234 345,123 234 345], keep order:false, stats:pseudo", + " └─TableRowIDScan_6(Probe) 0.00 cop[tikv] table:t28064 keep order:false, stats:pseudo")) +} + +func TestPreparePlanCache4Blacklist(t *testing.T) { + store, dom, err := newStoreWithBootstrap() + require.NoError(t, err) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + tk := testkit.NewTestKit(t, store) + defer func() { + dom.Close() + require.NoError(t, store.Close()) + }() + + tk.MustExec("use test") + tk.MustExec("set @@tidb_enable_collect_execution_info=0;") + + // test the blacklist of optimization rules + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int);") + tk.MustExec("prepare stmt from 'select min(a) from t;';") + tk.MustExec("execute stmt;") + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + res := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) + require.Regexp(t, ".*TopN.*", res.Rows()[1][0]) + + res = tk.MustQuery("explain format = 'brief' select min(a) from t") + require.Regexp(t, ".*TopN.*", res.Rows()[1][0]) + + tk.MustExec("INSERT INTO mysql.opt_rule_blacklist VALUES('max_min_eliminate');") + tk.MustExec("ADMIN reload opt_rule_blacklist;") + + tk.MustExec("execute stmt;") + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + tk.MustExec("execute stmt;") + tkProcess = tk.Session().ShowProcess() + ps = []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + res = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) + // Plans that have been cached will not be affected by the blacklist. + require.Regexp(t, ".*TopN.*", res.Rows()[1][0]) + + res = tk.MustQuery("explain format = 'brief' select min(a) from t") + require.Regexp(t, ".*StreamAgg.*", res.Rows()[0][0]) + + // test the blacklist of Expression Pushdown + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int);") + tk.MustExec("prepare stmt from 'SELECT * FROM t WHERE a < 2 and a > 2;';") + tk.MustExec("execute stmt;") + tkProcess = tk.Session().ShowProcess() + ps = []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + res = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) + require.Equal(t, 3, len(res.Rows())) + require.Regexp(t, ".*Selection.*", res.Rows()[1][0]) + require.Equal(t, "gt(test.t.a, 2), lt(test.t.a, 2)", res.Rows()[1][4]) + + res = tk.MustQuery("explain format = 'brief' SELECT * FROM t WHERE a < 2 and a > 2;") + require.Equal(t, 3, len(res.Rows())) + require.Equal(t, "gt(test.t.a, 2), lt(test.t.a, 2)", res.Rows()[1][4]) + + tk.MustExec("INSERT INTO mysql.expr_pushdown_blacklist VALUES('<','tikv','');") + tk.MustExec("ADMIN reload expr_pushdown_blacklist;") + + tk.MustExec("execute stmt;") + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + tk.MustExec("execute stmt;") + tkProcess = tk.Session().ShowProcess() + ps = []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + res = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) + // The expressions can still be pushed down to tikv. + require.Equal(t, 3, len(res.Rows())) + require.Regexp(t, ".*Selection.*", res.Rows()[1][0]) + require.Equal(t, "gt(test.t.a, 2), lt(test.t.a, 2)", res.Rows()[1][4]) + + res = tk.MustQuery("explain format = 'brief' SELECT * FROM t WHERE a < 2 and a > 2;") + require.Equal(t, 4, len(res.Rows())) + require.Regexp(t, ".*Selection.*", res.Rows()[0][0]) + require.Equal(t, "lt(test.t.a, 2)", res.Rows()[0][4]) + require.Regexp(t, ".*Selection.*", res.Rows()[2][0]) + require.Equal(t, "gt(test.t.a, 2)", res.Rows()[2][4]) + + tk.MustExec("DELETE FROM mysql.expr_pushdown_blacklist;") + tk.MustExec("ADMIN reload expr_pushdown_blacklist;") +} + +func TestPlanCacheClusterIndex(t *testing.T) { + store, dom, err := newStoreWithBootstrap() + require.NoError(t, err) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + tk := testkit.NewTestKit(t, store) + defer func() { + dom.Close() + require.NoError(t, store.Close()) + }() + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("set @@tidb_enable_collect_execution_info=0;") + tk.MustExec("create table t1(a varchar(20), b varchar(20), c varchar(20), primary key(a, b))") + tk.MustExec("insert into t1 values('1','1','111'),('2','2','222'),('3','3','333')") + + // For table scan + tk.MustExec(`prepare stmt1 from "select * from t1 where t1.a = ? and t1.b > ?"`) + tk.MustExec("set @v1 = '1'") + tk.MustExec("set @v2 = '0'") + tk.MustQuery("execute stmt1 using @v1,@v2").Check(testkit.Rows("1 1 111")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustExec("set @v1 = '2'") + tk.MustExec("set @v2 = '1'") + tk.MustQuery("execute stmt1 using @v1,@v2").Check(testkit.Rows("2 2 222")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustExec("set @v1 = '3'") + tk.MustExec("set @v2 = '2'") + tk.MustQuery("execute stmt1 using @v1,@v2").Check(testkit.Rows("3 3 333")) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + require.Equal(t, 0, strings.Index(rows[len(rows)-1][4].(string), `range:("3" "2","3" +inf]`)) + // For point get + tk.MustExec(`prepare stmt2 from "select * from t1 where t1.a = ? and t1.b = ?"`) + tk.MustExec("set @v1 = '1'") + tk.MustExec("set @v2 = '1'") + tk.MustQuery("execute stmt2 using @v1,@v2").Check(testkit.Rows("1 1 111")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustExec("set @v1 = '2'") + tk.MustExec("set @v2 = '2'") + tk.MustQuery("execute stmt2 using @v1,@v2").Check(testkit.Rows("2 2 222")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustExec("set @v1 = '3'") + tk.MustExec("set @v2 = '3'") + tk.MustQuery("execute stmt2 using @v1,@v2").Check(testkit.Rows("3 3 333")) + tkProcess = tk.Session().ShowProcess() + ps = []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + require.Equal(t, 0, strings.Index(rows[len(rows)-1][0].(string), `Point_Get`)) + // For CBO point get and batch point get + // case 1: + tk.MustExec(`drop table if exists ta, tb`) + tk.MustExec(`create table ta (a varchar(8) primary key, b int)`) + tk.MustExec(`insert ta values ('a', 1), ('b', 2)`) + tk.MustExec(`create table tb (a varchar(8) primary key, b int)`) + tk.MustExec(`insert tb values ('a', 1), ('b', 2)`) + tk.MustExec(`prepare stmt1 from "select * from ta, tb where ta.a = tb.a and ta.a = ?"`) + tk.MustExec(`set @v1 = 'a', @v2 = 'b'`) + tk.MustQuery(`execute stmt1 using @v1`).Check(testkit.Rows("a 1 a 1")) + tk.MustQuery(`execute stmt1 using @v2`).Check(testkit.Rows("b 2 b 2")) + + // case 2: + tk.MustExec(`drop table if exists ta, tb`) + tk.MustExec(`create table ta (a varchar(10) primary key, b int not null)`) + tk.MustExec(`insert ta values ('a', 1), ('b', 2)`) + tk.MustExec(`create table tb (b int primary key, c int)`) + tk.MustExec(`insert tb values (1, 1), (2, 2)`) + tk.MustExec(`prepare stmt1 from "select * from ta, tb where ta.b = tb.b and ta.a = ?"`) + tk.MustExec(`set @v1 = 'a', @v2 = 'b'`) + tk.MustQuery(`execute stmt1 using @v1`).Check(testkit.Rows("a 1 1 1")) + tk.MustQuery(`execute stmt1 using @v2`).Check(testkit.Rows("b 2 2 2")) + tk.MustQuery(`execute stmt1 using @v2`).Check(testkit.Rows("b 2 2 2")) + tkProcess = tk.Session().ShowProcess() + ps = []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + require.True(t, strings.Contains(rows[3][0].(string), `TableRangeScan`)) + + // case 3: + tk.MustExec(`drop table if exists ta, tb`) + tk.MustExec(`create table ta (a varchar(10), b varchar(10), c int, primary key (a, b))`) + tk.MustExec(`insert ta values ('a', 'a', 1), ('b', 'b', 2), ('c', 'c', 3)`) + tk.MustExec(`create table tb (b int primary key, c int)`) + tk.MustExec(`insert tb values (1, 1), (2, 2), (3,3)`) + tk.MustExec(`prepare stmt1 from "select * from ta, tb where ta.c = tb.b and ta.a = ? and ta.b = ?"`) + tk.MustExec(`set @v1 = 'a', @v2 = 'b', @v3 = 'c'`) + tk.MustQuery(`execute stmt1 using @v1, @v1`).Check(testkit.Rows("a a 1 1 1")) + tk.MustQuery(`execute stmt1 using @v2, @v2`).Check(testkit.Rows("b b 2 2 2")) + tk.MustExec(`prepare stmt2 from "select * from ta, tb where ta.c = tb.b and (ta.a, ta.b) in ((?, ?), (?, ?))"`) + tk.MustQuery(`execute stmt2 using @v1, @v1, @v2, @v2`).Check(testkit.Rows("a a 1 1 1", "b b 2 2 2")) + tk.MustQuery(`execute stmt2 using @v2, @v2, @v3, @v3`).Check(testkit.Rows("b b 2 2 2", "c c 3 3 3")) + + // For issue 19002 + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec(`drop table if exists t1`) + tk.MustExec(`create table t1(a int, b int, c int, primary key(a, b))`) + tk.MustExec(`insert into t1 values(1,1,111),(2,2,222),(3,3,333)`) + // Point Get: + tk.MustExec(`prepare stmt1 from "select * from t1 where t1.a = ? and t1.b = ?"`) + tk.MustExec(`set @v1=1, @v2=1`) + tk.MustQuery(`execute stmt1 using @v1,@v2`).Check(testkit.Rows("1 1 111")) + tk.MustExec(`set @v1=2, @v2=2`) + tk.MustQuery(`execute stmt1 using @v1,@v2`).Check(testkit.Rows("2 2 222")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + // Batch Point Get: + tk.MustExec(`prepare stmt2 from "select * from t1 where (t1.a,t1.b) in ((?,?),(?,?))"`) + tk.MustExec(`set @v1=1, @v2=1, @v3=2, @v4=2`) + tk.MustQuery(`execute stmt2 using @v1,@v2,@v3,@v4`).Check(testkit.Rows("1 1 111", "2 2 222")) + tk.MustExec(`set @v1=2, @v2=2, @v3=3, @v4=3`) + tk.MustQuery(`execute stmt2 using @v1,@v2,@v3,@v4`).Check(testkit.Rows("2 2 222", "3 3 333")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) +} + +func TestPlanCacheWithDifferentVariableTypes(t *testing.T) { + store, dom, err := newStoreWithBootstrap() + require.NoError(t, err) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + tk := testkit.NewTestKit(t, store) + defer func() { + dom.Close() + require.NoError(t, store.Close()) + }() + require.NoError(t, err) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("set @@tidb_enable_collect_execution_info=0;") + tk.MustExec("create table t1(a varchar(20), b int, c float, key(b, a))") + tk.MustExec("insert into t1 values('1',1,1.1),('2',2,222),('3',3,333)") + tk.MustExec("create table t2(a varchar(20), b int, c float, key(b, a))") + tk.MustExec("insert into t2 values('3',3,3.3),('2',2,222),('3',3,333)") + + var input []struct { + PrepareStmt string + Executes []struct { + Vars []struct { + Name string + Value string + } + ExecuteSQL string + } + } + var output []struct { + PrepareStmt string + Executes []struct { + SQL string + Vars []struct { + Name string + Value string + } + Plan []string + LastPlanUseCache string + Result []string + } + } + prepareMergeSuiteData.GetTestCases(t, &input, &output) + for i, tt := range input { + tk.MustExec(tt.PrepareStmt) + testdata.OnRecord(func() { + output[i].PrepareStmt = tt.PrepareStmt + output[i].Executes = make([]struct { + SQL string + Vars []struct { + Name string + Value string + } + Plan []string + LastPlanUseCache string + Result []string + }, len(tt.Executes)) + }) + require.Equal(t, tt.PrepareStmt, output[i].PrepareStmt) + for j, exec := range tt.Executes { + for _, v := range exec.Vars { + tk.MustExec(fmt.Sprintf(`set @%s = %s`, v.Name, v.Value)) + } + res := tk.MustQuery(exec.ExecuteSQL) + lastPlanUseCache := tk.MustQuery("select @@last_plan_from_cache").Rows()[0][0] + tk.MustQuery(exec.ExecuteSQL) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + plan := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) + testdata.OnRecord(func() { + output[i].Executes[j].SQL = exec.ExecuteSQL + output[i].Executes[j].Plan = testdata.ConvertRowsToStrings(plan.Rows()) + output[i].Executes[j].Vars = exec.Vars + output[i].Executes[j].LastPlanUseCache = lastPlanUseCache.(string) + output[i].Executes[j].Result = testdata.ConvertRowsToStrings(res.Rows()) + }) + + require.Equal(t, exec.ExecuteSQL, output[i].Executes[j].SQL) + plan.Check(testkit.Rows(output[i].Executes[j].Plan...)) + require.Equal(t, exec.Vars, output[i].Executes[j].Vars) + require.Equal(t, lastPlanUseCache.(string), output[i].Executes[j].LastPlanUseCache) + res.Check(testkit.Rows(output[i].Executes[j].Result...)) + } + } +} + +func TestPlanCacheOperators(t *testing.T) { + store, dom, err := newStoreWithBootstrap() + require.NoError(t, err) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + tk := testkit.NewTestKit(t, store) + defer func() { + dom.Close() + require.NoError(t, store.Close()) + }() + type ExecCase struct { + Parameters []string + UseCache bool + } + type PrepCase struct { + PrepStmt string + ExecCases []ExecCase + } + + cases := []PrepCase{ + {"use test", nil}, + + // cases for TableReader on PK + {"create table t (a int, b int, primary key(a))", nil}, + {"insert into t values (1,1), (2,2), (3,3), (4,4), (5,5), (6,null)", nil}, + {"select a from t where a=?", []ExecCase{ + {[]string{"1"}, false}, + {[]string{"2"}, true}, + {[]string{"3"}, true}, + }}, + {"select a from t where a in (?,?,?)", []ExecCase{ + {[]string{"1", "1", "1"}, false}, + {[]string{"2", "3", "4"}, true}, + {[]string{"3", "5", "7"}, true}, + }}, + {"select a from t where a>? and a? and a? and a? and a? and a?", []ExecCase{ + {[]string{"1"}, false}, + {[]string{"3"}, true}, + {[]string{"5"}, true}, + }}, + {"select /*+ HASH_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a and t2.b>?", []ExecCase{ + {[]string{"1"}, false}, + {[]string{"3"}, true}, + {[]string{"5"}, true}, + }}, + {"select /*+ HASH_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a and t1.b>? and t2.b?", []ExecCase{ + {[]string{"1"}, false}, + {[]string{"3"}, true}, + {[]string{"5"}, true}, + }}, + {"select /*+ MERGE_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a and t2.b>?", []ExecCase{ + {[]string{"1"}, false}, + {[]string{"3"}, true}, + {[]string{"5"}, true}, + }}, + {"select /*+ MERGE_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a and t1.b>? and t2.b?", []ExecCase{ + {[]string{"1"}, false}, + {[]string{"3"}, true}, + {[]string{"5"}, true}, + }}, + {"select /*+ INL_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a and t2.b>?", []ExecCase{ + {[]string{"1"}, false}, + {[]string{"3"}, true}, + {[]string{"5"}, true}, + }}, + {"select /*+ INL_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a and t1.b>? and t2.b? and t1.a > (select min(t2.a) from t t2 where t2.b < t1.b)", []ExecCase{ + {[]string{"1"}, false}, + {[]string{"3"}, false}, // plans with sub-queries cannot be cached, but the result must be correct + {[]string{"5"}, false}, + }}, + {"select * from t t1 where t1.a > (select min(t2.a) from t t2 where t2.b < t1.b+?)", []ExecCase{ + {[]string{"1"}, false}, + {[]string{"3"}, false}, + {[]string{"5"}, false}, + }}, + {"select * from t t1 where t1.b>? and t1.a > (select min(t2.a) from t t2 where t2.b < t1.b+?)", []ExecCase{ + {[]string{"1", "1"}, false}, + {[]string{"3", "2"}, false}, + {[]string{"5", "3"}, false}, + }}, + {"drop table t", nil}, + + // cases for Window + {"create table t (name varchar(50), y int, sale decimal(14,2))", nil}, + {"insert into t values ('Bob',2016,2.4), ('Bob',2017,3.2), ('Bob',2018,2.1), ('Alice',2016,1.4), ('Alice',2017,2), ('Alice',2018,3.3), ('John',2016,4), ('John',2017,2.1), ('John',2018,5)", nil}, + {"select *, sum(sale) over (partition by y order by sale) total from t where sale>? order by y", []ExecCase{ + {[]string{"0.1"}, false}, + {[]string{"0.5"}, true}, + {[]string{"1.5"}, true}, + {[]string{"3.5"}, true}, + }}, + {"select *, sum(sale) over (partition by y order by sale+? rows 2 preceding) total from t order by y", []ExecCase{ + {[]string{"0.1"}, false}, + {[]string{"0.5"}, true}, + {[]string{"1.5"}, true}, + {[]string{"3.5"}, true}, + }}, + {"select *, rank() over (partition by y order by sale+? rows 2 preceding) total from t order by y", []ExecCase{ + {[]string{"0.1"}, false}, + {[]string{"0.5"}, true}, + {[]string{"1.5"}, true}, + {[]string{"3.5"}, true}, + }}, + {"select *, first_value(sale) over (partition by y order by sale+? rows 2 preceding) total from t order by y", []ExecCase{ + {[]string{"0.1"}, false}, + {[]string{"0.5"}, true}, + {[]string{"1.5"}, true}, + {[]string{"3.5"}, true}, + }}, + {"select *, first_value(sale) over (partition by y order by sale rows ? preceding) total from t order by y", []ExecCase{ + {[]string{"1"}, false}, // window plans with parameters in frame cannot be cached + {[]string{"2"}, false}, + {[]string{"3"}, false}, + {[]string{"4"}, false}, + }}, + {"drop table t", nil}, + + // cases for Limit + {"create table t (a int)", nil}, + {"insert into t values (1), (1), (2), (2), (3), (4), (5), (6), (7), (8), (9), (0), (0)", nil}, + {"select * from t limit ?", []ExecCase{ + {[]string{"20"}, false}, + {[]string{"30"}, false}, + }}, + {"select * from t limit 40, ?", []ExecCase{ + {[]string{"1"}, false}, + {[]string{"2"}, false}, + }}, + {"select * from t limit ?, 10", []ExecCase{ + {[]string{"20"}, false}, + {[]string{"30"}, false}, + }}, + {"select * from t limit ?, ?", []ExecCase{ + {[]string{"20", "20"}, false}, + {[]string{"20", "40"}, false}, + }}, + {"select * from t where a? order by mod(a, 3)", []ExecCase{ + {[]string{"1"}, false}, + {[]string{"2"}, true}, + {[]string{"3"}, true}, + }}, + + // cases for topN + {"select * from t order by b limit ?", []ExecCase{ + {[]string{"1"}, false}, + {[]string{"2"}, false}, + }}, + {"select * from t order by b limit 10, ?", []ExecCase{ + {[]string{"1"}, false}, + {[]string{"2"}, false}, + }}, + {"select * from t order by ? limit 10", []ExecCase{ + {[]string{"1"}, false}, + {[]string{"2"}, false}, + }}, + {"select * from t order by ? limit ?", []ExecCase{ + {[]string{"1", "10"}, false}, + {[]string{"2", "20"}, false}, + }}, + } + + for _, prepCase := range cases { + isQuery := strings.Contains(prepCase.PrepStmt, "select") + if !isQuery { + tk.MustExec(prepCase.PrepStmt) + continue + } + + tk.MustExec(fmt.Sprintf(`prepare stmt from '%v'`, prepCase.PrepStmt)) + for _, execCase := range prepCase.ExecCases { + // set all parameters + usingStmt := "" + if len(execCase.Parameters) > 0 { + setStmt := "set " + usingStmt = "using " + for i, parameter := range execCase.Parameters { + if i > 0 { + setStmt += ", " + usingStmt += ", " + } + setStmt += fmt.Sprintf("@x%v=%v", i, parameter) + usingStmt += fmt.Sprintf("@x%v", i) + } + tk.MustExec(setStmt) + } + + // execute this statement and check whether it uses a cached plan + results := tk.MustQuery("execute stmt " + usingStmt).Sort().Rows() + useCache := "0" + if execCase.UseCache { + useCache = "1" + } + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(useCache)) + + // check whether the result is correct + tmp := strings.Split(prepCase.PrepStmt, "?") + require.Equal(t, len(execCase.Parameters)+1, len(tmp)) + query := "" + for i := range tmp { + query += tmp[i] + if i < len(execCase.Parameters) { + query += execCase.Parameters[i] + } + } + tk.MustQuery(query).Sort().Check(results) + } + } +} + +func TestIssue28782(t *testing.T) { + store, dom, err := newStoreWithBootstrap() + require.NoError(t, err) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + tk := testkit.NewTestKit(t, store) + defer func() { + dom.Close() + require.NoError(t, store.Close()) + }() + tk.MustExec("use test") + tk.MustExec("set @@tidb_enable_collect_execution_info=0;") + tk.MustExec("prepare stmt from 'SELECT IF(?, 1, 0);';") + tk.MustExec("set @a=1, @b=null, @c=0") + + tk.MustQuery("execute stmt using @a;").Check(testkit.Rows("1")) + tk.MustQuery("execute stmt using @b;").Check(testkit.Rows("0")) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + tk.MustQuery("execute stmt using @c;").Check(testkit.Rows("0")) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) +} + +func TestIssue29101(t *testing.T) { + store, dom, err := newStoreWithBootstrap() + require.NoError(t, err) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + tk := testkit.NewTestKit(t, store) + defer func() { + dom.Close() + require.NoError(t, store.Close()) + }() + + tk.MustExec(`use test`) + tk.MustExec("set @@tidb_enable_collect_execution_info=0;") + tk.MustExec(`CREATE TABLE customer ( + c_id int(11) NOT NULL, + c_d_id int(11) NOT NULL, + c_w_id int(11) NOT NULL, + c_first varchar(16) DEFAULT NULL, + c_last varchar(16) DEFAULT NULL, + c_credit char(2) DEFAULT NULL, + c_discount decimal(4,4) DEFAULT NULL, + PRIMARY KEY (c_w_id,c_d_id,c_id), + KEY idx_customer (c_w_id,c_d_id,c_last,c_first) + )`) + tk.MustExec(`CREATE TABLE warehouse ( + w_id int(11) NOT NULL, + w_tax decimal(4,4) DEFAULT NULL, + PRIMARY KEY (w_id) + )`) + tk.MustExec(`prepare s1 from 'SELECT /*+ TIDB_INLJ(customer,warehouse) */ c_discount, c_last, c_credit, w_tax FROM customer, warehouse WHERE w_id = ? AND c_w_id = w_id AND c_d_id = ? AND c_id = ?'`) + tk.MustExec(`set @a=936,@b=7,@c=158`) + tk.MustQuery(`execute s1 using @a,@b,@c`).Check(testkit.Rows()) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // can use IndexJoin + `Projection_6 1.00 root test.customer.c_discount, test.customer.c_last, test.customer.c_credit, test.warehouse.w_tax`, + `└─IndexJoin_14 1.00 root inner join, inner:TableReader_10, outer key:test.customer.c_w_id, inner key:test.warehouse.w_id, equal cond:eq(test.customer.c_w_id, test.warehouse.w_id)`, + ` ├─Point_Get_33(Build) 1.00 root table:customer, index:PRIMARY(c_w_id, c_d_id, c_id) `, + ` └─TableReader_10(Probe) 0.00 root data:Selection_9`, + ` └─Selection_9 0.00 cop[tikv] eq(test.warehouse.w_id, 936)`, + ` └─TableRangeScan_8 1.00 cop[tikv] table:warehouse range: decided by [test.customer.c_w_id], keep order:false, stats:pseudo`)) + tk.MustQuery(`execute s1 using @a,@b,@c`).Check(testkit.Rows()) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) // can use the plan-cache + + tk.MustExec(`CREATE TABLE order_line ( + ol_o_id int(11) NOT NULL, + ol_d_id int(11) NOT NULL, + ol_w_id int(11) NOT NULL, + ol_number int(11) NOT NULL, + ol_i_id int(11) NOT NULL, + PRIMARY KEY (ol_w_id,ol_d_id,ol_o_id,ol_number))`) + tk.MustExec(`CREATE TABLE stock ( + s_i_id int(11) NOT NULL, + s_w_id int(11) NOT NULL, + s_quantity int(11) DEFAULT NULL, + PRIMARY KEY (s_w_id,s_i_id))`) + tk.MustExec(`prepare s1 from 'SELECT /*+ TIDB_INLJ(order_line,stock) */ COUNT(DISTINCT (s_i_id)) stock_count FROM order_line, stock WHERE ol_w_id = ? AND ol_d_id = ? AND ol_o_id < ? AND ol_o_id >= ? - 20 AND s_w_id = ? AND s_i_id = ol_i_id AND s_quantity < ?'`) + tk.MustExec(`set @a=391,@b=1,@c=3058,@d=18`) + tk.MustExec(`execute s1 using @a,@b,@c,@c,@a,@d`) + tkProcess = tk.Session().ShowProcess() + ps = []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // can use index-join + `StreamAgg_9 1.00 root funcs:count(distinct test.stock.s_i_id)->Column#11`, + `└─IndexJoin_14 0.03 root inner join, inner:IndexLookUp_13, outer key:test.order_line.ol_i_id, inner key:test.stock.s_i_id, equal cond:eq(test.order_line.ol_i_id, test.stock.s_i_id)`, + ` ├─Selection_30(Build) 0.03 root eq(test.order_line.ol_d_id, 1), eq(test.order_line.ol_w_id, 391), ge(test.order_line.ol_o_id, 3038), lt(test.order_line.ol_o_id, 3058)`, + ` │ └─IndexLookUp_29 0.03 root `, + ` │ ├─IndexRangeScan_27(Build) 0.03 cop[tikv] table:order_line, index:PRIMARY(ol_w_id, ol_d_id, ol_o_id, ol_number) range:[391 1 3038,391 1 3058), keep order:false, stats:pseudo`, + ` │ └─TableRowIDScan_28(Probe) 0.03 cop[tikv] table:order_line keep order:false, stats:pseudo`, + ` └─IndexLookUp_13(Probe) 1.00 root `, + ` ├─IndexRangeScan_10(Build) 1.00 cop[tikv] table:stock, index:PRIMARY(s_w_id, s_i_id) range: decided by [eq(test.stock.s_i_id, test.order_line.ol_i_id) eq(test.stock.s_w_id, 391)], keep order:false, stats:pseudo`, + ` └─Selection_12(Probe) 1.00 cop[tikv] lt(test.stock.s_quantity, 18)`, + ` └─TableRowIDScan_11 1.00 cop[tikv] table:stock keep order:false, stats:pseudo`)) + tk.MustExec(`execute s1 using @a,@b,@c,@c,@a,@d`) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) // can use the plan-cache +} + +func TestIssue28087And28162(t *testing.T) { + store, dom, err := newStoreWithBootstrap() + require.NoError(t, err) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + tk := testkit.NewTestKit(t, store) + defer func() { + dom.Close() + require.NoError(t, store.Close()) + }() + + // issue 28087 + tk.MustExec(`use test`) + tk.MustExec(`drop table if exists IDT_26207`) + tk.MustExec(`CREATE TABLE IDT_26207 (col1 bit(1))`) + tk.MustExec(`insert into IDT_26207 values(0x0), (0x1)`) + tk.MustExec(`prepare stmt from 'select t1.col1 from IDT_26207 as t1 left join IDT_26207 as t2 on t1.col1 = t2.col1 where t1.col1 in (?, ?, ?)'`) + tk.MustExec(`set @a=0x01, @b=0x01, @c=0x01`) + tk.MustQuery(`execute stmt using @a,@b,@c`).Check(testkit.Rows("\x01")) + tk.MustExec(`set @a=0x00, @b=0x00, @c=0x01`) + tk.MustQuery(`execute stmt using @a,@b,@c`).Check(testkit.Rows("\x00", "\x01")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + + // issue 28162 + tk.MustExec(`drop table if exists IDT_MC21780`) + tk.MustExec(`CREATE TABLE IDT_MC21780 ( + COL1 timestamp NULL DEFAULT NULL, + COL2 timestamp NULL DEFAULT NULL, + COL3 timestamp NULL DEFAULT NULL, + KEY U_M_COL (COL1,COL2) + )`) + tk.MustExec(`insert into IDT_MC21780 values("1970-12-18 10:53:28", "1970-12-18 10:53:28", "1970-12-18 10:53:28")`) + tk.MustExec(`prepare stmt from 'select/*+ hash_join(t1) */ * from IDT_MC21780 t1 join IDT_MC21780 t2 on t1.col1 = t2.col1 where t1. col1 < ? and t2. col1 in (?, ?, ?);'`) + tk.MustExec(`set @a="2038-01-19 03:14:07", @b="2038-01-19 03:14:07", @c="2038-01-19 03:14:07", @d="2038-01-19 03:14:07"`) + tk.MustQuery(`execute stmt using @a,@b,@c,@d`).Check(testkit.Rows()) + tk.MustExec(`set @a="1976-09-09 20:21:11", @b="2021-07-14 09:28:16", @c="1982-01-09 03:36:39", @d="1970-12-18 10:53:28"`) + tk.MustQuery(`execute stmt using @a,@b,@c,@d`).Check(testkit.Rows("1970-12-18 10:53:28 1970-12-18 10:53:28 1970-12-18 10:53:28 1970-12-18 10:53:28 1970-12-18 10:53:28 1970-12-18 10:53:28")) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) +} + +func TestParameterPushDown(t *testing.T) { + store, dom, err := newStoreWithBootstrap() + require.NoError(t, err) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + tk := testkit.NewTestKit(t, store) + defer func() { + dom.Close() + require.NoError(t, store.Close()) + }() + require.NoError(t, err) + tk.MustExec(`use test`) + tk.MustExec(`drop table if exists t`) + tk.MustExec(`create table t (a int, b int, c int, key(a))`) + tk.MustExec(`insert into t values (1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5), (6, 6, 6)`) + tk.MustExec("set @@tidb_enable_collect_execution_info=0;") + tk.MustExec(`set @x1=1,@x5=5,@x10=10,@x20=20`) + + var input []struct { + SQL string + } + var output []struct { + Result []string + Plan []string + FromCache string + } + prepareMergeSuiteData.GetTestCases(t, &input, &output) + + for i, tt := range input { + if strings.HasPrefix(tt.SQL, "execute") { + res := tk.MustQuery(tt.SQL).Sort() + fromCache := tk.MustQuery("select @@last_plan_from_cache") + tk.MustQuery(tt.SQL) + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + plan := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) + + testdata.OnRecord(func() { + output[i].Result = testdata.ConvertRowsToStrings(res.Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) + output[i].FromCache = fromCache.Rows()[0][0].(string) + }) + + res.Check(testkit.Rows(output[i].Result...)) + plan.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, fromCache.Rows()[0][0].(string), output[i].FromCache) + } else { + tk.MustExec(tt.SQL) + testdata.OnRecord(func() { + output[i].Result = nil + }) + } + } +} + +func TestPreparePlanCache4Function(t *testing.T) { + store, dom, err := newStoreWithBootstrap() + require.NoError(t, err) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + tk := testkit.NewTestKit(t, store) + defer func() { + dom.Close() + require.NoError(t, store.Close()) + }() + + tk.MustExec("use test") + tk.MustExec("set @@tidb_enable_collect_execution_info=0;") + + // Testing for non-deterministic functions + tk.MustExec("prepare stmt from 'select rand()';") + res := tk.MustQuery("execute stmt;") + require.Equal(t, 1, len(res.Rows())) + + res1 := tk.MustQuery("execute stmt;") + require.Equal(t, 1, len(res1.Rows())) + require.NotEqual(t, res.Rows()[0][0], res1.Rows()[0][0]) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + + // Testing for control functions + tk.MustExec("prepare stmt from 'SELECT IFNULL(?,0);';") + tk.MustExec("set @a = 1, @b = null;") + tk.MustQuery("execute stmt using @a;").Check(testkit.Rows("1")) + tk.MustQuery("execute stmt using @b;").Check(testkit.Rows("0")) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int);") + tk.MustExec("prepare stmt from 'select a, case when a = ? then 0 when a <=> ? then 1 else 2 end b from t order by a;';") + tk.MustExec("insert into t values(0), (1), (2), (null);") + tk.MustExec("set @a = 0, @b = 1, @c = 2, @d = null;") + tk.MustQuery("execute stmt using @a, @b;").Check(testkit.Rows(" 2", "0 0", "1 1", "2 2")) + tk.MustQuery("execute stmt using @c, @d;").Check(testkit.Rows(" 1", "0 2", "1 2", "2 0")) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) +} + +func TestPreparePlanCache4DifferentSystemVars(t *testing.T) { + store, dom, err := newStoreWithBootstrap() + require.NoError(t, err) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + tk := testkit.NewTestKit(t, store) + defer func() { + dom.Close() + require.NoError(t, store.Close()) + }() + + tk.MustExec("use test") + tk.MustExec("set @@tidb_enable_collect_execution_info=0;") + + // Testing for 'sql_select_limit' + tk.MustExec("set @@sql_select_limit = 1") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(0), (1), (null);") + tk.MustExec("prepare stmt from 'select a from t order by a;';") + tk.MustQuery("execute stmt;").Check(testkit.Rows("")) + + tk.MustExec("set @@sql_select_limit = 2") + tk.MustQuery("execute stmt;").Check(testkit.Rows("", "0")) + // The 'sql_select_limit' will be stored in the cache key. So if the `sql_select_limit` + // have been changed, the plan cache can not be reused. + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) + + tk.MustExec("set @@sql_select_limit = 18446744073709551615") + tk.MustQuery("execute stmt;").Check(testkit.Rows("", "0", "1")) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) + + // test for 'tidb_enable_index_merge' + tk.MustExec("set @@tidb_enable_index_merge = 1;") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int, b int, index idx_a(a), index idx_b(b));") + tk.MustExec("prepare stmt from 'select * from t use index(idx_a, idx_b) where a > 1 or b > 1;';") + tk.MustExec("execute stmt;") + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + res := tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) + require.Equal(t, 4, len(res.Rows())) + require.Regexp(t, ".*IndexMerge.*", res.Rows()[0][0]) + + tk.MustExec("set @@tidb_enable_index_merge = 0;") + tk.MustExec("execute stmt;") + tkProcess = tk.Session().ShowProcess() + ps = []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) + require.Equal(t, 4, len(res.Rows())) + require.Regexp(t, ".*IndexMerge.*", res.Rows()[0][0]) + tk.MustExec("execute stmt;") + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + + // test for 'tidb_enable_parallel_apply' + tk.MustExec("set @@tidb_enable_collect_execution_info=1;") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int)") + tk.MustExec("insert into t values (0, 0), (1, 1), (2, 2), (3, 3), (4, 4), (5, 5), (6, 6), (7, 7), (8, 8), (9, 9), (null, null)") + + tk.MustExec("set tidb_enable_parallel_apply=true") + tk.MustExec("prepare stmt from 'select t1.b from t t1 where t1.b > (select max(b) from t t2 where t1.a > t2.a);';") + tk.MustQuery("execute stmt;").Sort().Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7", "8", "9")) + tkProcess = tk.Session().ShowProcess() + ps = []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) + require.Regexp(t, ".*Apply.*", res.Rows()[1][0]) + require.Regexp(t, ".*Concurrency.*", res.Rows()[1][5]) + + tk.MustExec("set tidb_enable_parallel_apply=false") + tk.MustQuery("execute stmt;").Sort().Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7", "8", "9")) + tkProcess = tk.Session().ShowProcess() + ps = []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) + require.Regexp(t, ".*Apply.*", res.Rows()[1][0]) + executionInfo := fmt.Sprintf("%v", res.Rows()[1][4]) + // Do not use the parallel apply. + require.False(t, strings.Contains(executionInfo, "Concurrency")) + tk.MustExec("execute stmt;") + // The subquery plan can not be cached. + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) + + // test for apply cache + tk.MustExec("set @@tidb_enable_collect_execution_info=1;") + tk.MustExec("set tidb_mem_quota_apply_cache=33554432") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int)") + tk.MustExec("insert into t values (0, 0), (1, 1), (2, 2), (3, 3), (4, 4), (5, 5), (6, 6), (7, 7), (8, 8), (9, 9), (null, null)") + + tk.MustExec("prepare stmt from 'select t1.b from t t1 where t1.b > (select max(b) from t t2 where t1.a > t2.a);';") + tk.MustQuery("execute stmt;").Sort().Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7", "8", "9")) + tkProcess = tk.Session().ShowProcess() + ps = []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) + require.Regexp(t, ".*Apply.*", res.Rows()[1][0]) + require.Regexp(t, ".*cache:ON.*", res.Rows()[1][5]) + + tk.MustExec("set tidb_mem_quota_apply_cache=0") + tk.MustQuery("execute stmt;").Sort().Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7", "8", "9")) + tkProcess = tk.Session().ShowProcess() + ps = []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) + require.Regexp(t, ".*Apply.*", res.Rows()[1][0]) + executionInfo = fmt.Sprintf("%v", res.Rows()[1][5]) + // Do not use the apply cache. + require.True(t, strings.Contains(executionInfo, "cache:OFF")) + tk.MustExec("execute stmt;") + // The subquery plan can not be cached. + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) +} + +func TestTemporaryTable4PlanCache(t *testing.T) { + store, dom, err := newStoreWithBootstrap() + require.NoError(t, err) + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(true) + tk := testkit.NewTestKit(t, store) + defer func() { + dom.Close() + require.NoError(t, store.Close()) + }() + tk.MustExec("use test") + tk.MustExec("set @@tidb_enable_collect_execution_info=0;") + tk.MustExec("drop table if exists tmp2") + tk.MustExec("create temporary table tmp2 (a int, b int, key(a), key(b));") + tk.MustExec("prepare stmt from 'select * from tmp2;';") + tk.MustQuery("execute stmt;").Check(testkit.Rows()) + tk.MustQuery("execute stmt;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) + + tk.MustExec("drop table if exists tmp_t;") + tk.MustExec("create global temporary table tmp_t (id int primary key, a int, b int, index(a)) on commit delete rows") + tk.MustExec("prepare stmt from 'select * from tmp_t;';") + tk.MustQuery("execute stmt;").Check(testkit.Rows()) + tk.MustQuery("execute stmt;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) + +} + +func TestPrepareStmtAfterIsolationReadChange(t *testing.T) { + if israce.RaceEnabled { + t.Skip("race test for this case takes too long time") + } + store, clean := testkit.CreateMockStore(t) + defer clean() + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + plannercore.SetPreparedPlanCache(false) // requires plan cache disabled + tk := testkit.NewTestKit(t, store) + tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost", CurrentUser: true, AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + // create virtual tiflash replica. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec("set @@session.tidb_isolation_read_engines='tikv'") + tk.MustExec("set @@tidb_enable_collect_execution_info=0;") + tk.MustExec("prepare stmt from \"select * from t\"") + tk.MustQuery("execute stmt") + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + require.Equal(t, "cop[tikv]", rows[len(rows)-1][2]) + + tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash'") + tk.MustExec("execute stmt") + tkProcess = tk.Session().ShowProcess() + ps = []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() + require.Equal(t, rows[len(rows)-1][2], "cop[tiflash]") + + require.Equal(t, 1, len(tk.Session().GetSessionVars().PreparedStmts)) + require.Equal(t, "select * from `t`", tk.Session().GetSessionVars().PreparedStmts[1].(*plannercore.CachedPrepareStmt).NormalizedSQL) + require.Equal(t, "", tk.Session().GetSessionVars().PreparedStmts[1].(*plannercore.CachedPrepareStmt).NormalizedPlan) +} diff --git a/executor/prepared_test.go b/executor/prepared_test.go index da227f7585ea6..7117e6500ee66 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -17,48 +17,51 @@ package executor_test import ( "crypto/tls" "fmt" - "strconv" - "strings" "sync/atomic" + "testing" - . "github.com/pingcap/check" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/parser/auth" - "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" txninfo "github.com/pingcap/tidb/session/txninfo" - "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/israce" - "github.com/pingcap/tidb/util/testkit" + "github.com/stretchr/testify/require" ) -func (s *testSuite1) TestPreparedNameResolver(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPreparedNameResolver(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 t") tk.MustExec("create table t (id int, KEY id (id))") tk.MustExec("prepare stmt from 'select * from t limit ? offset ?'") _, err := tk.Exec("prepare stmt from 'select b from t'") - c.Assert(err.Error(), Equals, "[planner:1054]Unknown column 'b' in 'field list'") + require.EqualError(t, err, "[planner:1054]Unknown column 'b' in 'field list'") _, err = tk.Exec("prepare stmt from '(select * FROM t) union all (select * FROM t) order by a limit ?'") - c.Assert(err.Error(), Equals, "[planner:1054]Unknown column 'a' in 'order clause'") + require.EqualError(t, err, "[planner:1054]Unknown column 'a' in 'order clause'") } // a 'create table' DDL statement should be accepted if it has no parameters. -func (s *testSuite1) TestPreparedDDL(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPreparedDDL(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 t") tk.MustExec("prepare stmt from 'create table t (id int, KEY id (id))'") } // TestUnsupportedStmtForPrepare is related to https://github.com/pingcap/tidb/issues/17412 -func (s *testSuite1) TestUnsupportedStmtForPrepare(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUnsupportedStmtForPrepare(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`prepare stmt0 from "create table t0(a int primary key)"`) tk.MustGetErrCode(`prepare stmt1 from "execute stmt0"`, mysql.ErrUnsupportedPs) @@ -66,8 +69,11 @@ func (s *testSuite1) TestUnsupportedStmtForPrepare(c *C) { tk.MustGetErrCode(`prepare stmt4 from "prepare stmt3 from 'create table t1(a int, b int)'"`, mysql.ErrUnsupportedPs) } -func (s *testSuite1) TestIgnorePlanCache(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIgnorePlanCache(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 t") @@ -78,59 +84,7 @@ func (s *testSuite1) TestIgnorePlanCache(c *C) { tk.MustExec("prepare stmt from 'select /*+ IGNORE_PLAN_CACHE() */ * from t where id=?'") tk.MustExec("set @ignore_plan_doma = 1") tk.MustExec("execute stmt using @ignore_plan_doma") - c.Assert(tk.Se.GetSessionVars().StmtCtx.UseCache, IsFalse) -} - -func (s *testSerialSuite) TestPrepareStmtAfterIsolationReadChange(c *C) { - if israce.RaceEnabled { - c.Skip("race test for this case takes too long time") - } - tk := testkit.NewTestKitWithInit(c, s.store) - tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost", CurrentUser: true, AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) - - orgEnable := plannercore.PreparedPlanCacheEnabled() - defer func() { - plannercore.SetPreparedPlanCache(orgEnable) - }() - plannercore.SetPreparedPlanCache(false) // requires plan cache disabled - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int)") - // create virtual tiflash replica. - dom := domain.GetDomain(tk.Se) - is := dom.InfoSchema() - db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) - for _, tblInfo := range db.Tables { - if tblInfo.Name.L == "t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@session.tidb_isolation_read_engines='tikv'") - tk.MustExec("set @@tidb_enable_collect_execution_info=0;") - tk.MustExec("prepare stmt from \"select * from t\"") - tk.MustQuery("execute stmt") - tkProcess := tk.Se.ShowProcess() - ps := []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() - c.Assert(rows[len(rows)-1][2], Equals, "cop[tikv]") - - tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash'") - tk.MustExec("execute stmt") - tkProcess = tk.Se.ShowProcess() - ps = []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() - c.Assert(rows[len(rows)-1][2], Equals, "cop[tiflash]") - - c.Assert(len(tk.Se.GetSessionVars().PreparedStmts), Equals, 1) - c.Assert(tk.Se.GetSessionVars().PreparedStmts[1].(*plannercore.CachedPrepareStmt).NormalizedSQL, Equals, "select * from `t`") - c.Assert(tk.Se.GetSessionVars().PreparedStmts[1].(*plannercore.CachedPrepareStmt).NormalizedPlan, Equals, "") + require.False(t, tk.Session().GetSessionVars().StmtCtx.UseCache) } type mockSessionManager2 struct { @@ -157,6 +111,7 @@ func (sm *mockSessionManager2) GetProcessInfo(id uint64) (pi *util.ProcessInfo, } return } + func (sm *mockSessionManager2) Kill(connectionID uint64, query bool) { atomic.StoreInt32(&sm.killed, 1) atomic.StoreUint32(&sm.se.GetSessionVars().Killed, 1) @@ -167,25 +122,20 @@ func (sm *mockSessionManager2) ServerID() uint64 { return 1 } -var _ = SerialSuites(&testSuite12{&baseTestSuite{}}) - -type testSuite12 struct { - *baseTestSuite -} - -func (s *testSuite12) TestPreparedStmtWithHint(c *C) { +func TestPreparedStmtWithHint(t *testing.T) { + t.Parallel() // see https://github.com/pingcap/tidb/issues/18535 store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { - store.Close() + require.NoError(t, store.Close()) dom.Close() }() se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - tk.Se = se + require.NoError(t, err) + tk := testkit.NewTestKit(t, store) + tk.SetSession(se) sm := &mockSessionManager2{ se: se, @@ -194,1103 +144,82 @@ func (s *testSuite12) TestPreparedStmtWithHint(c *C) { go dom.ExpensiveQueryHandle().SetSessionManager(sm).Run() tk.MustExec("prepare stmt from \"select /*+ max_execution_time(100) */ sleep(10)\"") tk.MustQuery("execute stmt").Check(testkit.Rows("1")) - c.Check(atomic.LoadInt32(&sm.killed), Equals, int32(1)) -} - -func (s *testSerialSuite) TestPlanCacheClusterIndex(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - orgEnable := plannercore.PreparedPlanCacheEnabled() - defer func() { - plannercore.SetPreparedPlanCache(orgEnable) - }() - plannercore.SetPreparedPlanCache(true) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec("set @@tidb_enable_collect_execution_info=0;") - tk.MustExec("create table t1(a varchar(20), b varchar(20), c varchar(20), primary key(a, b))") - tk.MustExec("insert into t1 values('1','1','111'),('2','2','222'),('3','3','333')") - - // For table scan - tk.MustExec(`prepare stmt1 from "select * from t1 where t1.a = ? and t1.b > ?"`) - tk.MustExec("set @v1 = '1'") - tk.MustExec("set @v2 = '0'") - tk.MustQuery("execute stmt1 using @v1,@v2").Check(testkit.Rows("1 1 111")) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) - tk.MustExec("set @v1 = '2'") - tk.MustExec("set @v2 = '1'") - tk.MustQuery("execute stmt1 using @v1,@v2").Check(testkit.Rows("2 2 222")) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) - tk.MustExec("set @v1 = '3'") - tk.MustExec("set @v2 = '2'") - tk.MustQuery("execute stmt1 using @v1,@v2").Check(testkit.Rows("3 3 333")) - tkProcess := tk.Se.ShowProcess() - ps := []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() - c.Assert(strings.Index(rows[len(rows)-1][4].(string), `range:("3" "2","3" +inf]`), Equals, 0) - - // For point get - tk.MustExec(`prepare stmt2 from "select * from t1 where t1.a = ? and t1.b = ?"`) - tk.MustExec("set @v1 = '1'") - tk.MustExec("set @v2 = '1'") - tk.MustQuery("execute stmt2 using @v1,@v2").Check(testkit.Rows("1 1 111")) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) - tk.MustExec("set @v1 = '2'") - tk.MustExec("set @v2 = '2'") - tk.MustQuery("execute stmt2 using @v1,@v2").Check(testkit.Rows("2 2 222")) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) - tk.MustExec("set @v1 = '3'") - tk.MustExec("set @v2 = '3'") - tk.MustQuery("execute stmt2 using @v1,@v2").Check(testkit.Rows("3 3 333")) - tkProcess = tk.Se.ShowProcess() - ps = []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() - c.Assert(strings.Index(rows[len(rows)-1][0].(string), `Point_Get`), Equals, 0) - - // For CBO point get and batch point get - // case 1: - tk.MustExec(`drop table if exists ta, tb`) - tk.MustExec(`create table ta (a varchar(8) primary key, b int)`) - tk.MustExec(`insert ta values ('a', 1), ('b', 2)`) - tk.MustExec(`create table tb (a varchar(8) primary key, b int)`) - tk.MustExec(`insert tb values ('a', 1), ('b', 2)`) - tk.MustExec(`prepare stmt1 from "select * from ta, tb where ta.a = tb.a and ta.a = ?"`) - tk.MustExec(`set @v1 = 'a', @v2 = 'b'`) - tk.MustQuery(`execute stmt1 using @v1`).Check(testkit.Rows("a 1 a 1")) - tk.MustQuery(`execute stmt1 using @v2`).Check(testkit.Rows("b 2 b 2")) - - // case 2: - tk.MustExec(`drop table if exists ta, tb`) - tk.MustExec(`create table ta (a varchar(10) primary key, b int not null)`) - tk.MustExec(`insert ta values ('a', 1), ('b', 2)`) - tk.MustExec(`create table tb (b int primary key, c int)`) - tk.MustExec(`insert tb values (1, 1), (2, 2)`) - tk.MustExec(`prepare stmt1 from "select * from ta, tb where ta.b = tb.b and ta.a = ?"`) - tk.MustExec(`set @v1 = 'a', @v2 = 'b'`) - tk.MustQuery(`execute stmt1 using @v1`).Check(testkit.Rows("a 1 1 1")) - tk.MustQuery(`execute stmt1 using @v2`).Check(testkit.Rows("b 2 2 2")) - tk.MustQuery(`execute stmt1 using @v2`).Check(testkit.Rows("b 2 2 2")) - tkProcess = tk.Se.ShowProcess() - ps = []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Rows() - c.Assert(strings.Contains(rows[3][0].(string), `TableRangeScan`), IsTrue) - - // case 3: - tk.MustExec(`drop table if exists ta, tb`) - tk.MustExec(`create table ta (a varchar(10), b varchar(10), c int, primary key (a, b))`) - tk.MustExec(`insert ta values ('a', 'a', 1), ('b', 'b', 2), ('c', 'c', 3)`) - tk.MustExec(`create table tb (b int primary key, c int)`) - tk.MustExec(`insert tb values (1, 1), (2, 2), (3,3)`) - tk.MustExec(`prepare stmt1 from "select * from ta, tb where ta.c = tb.b and ta.a = ? and ta.b = ?"`) - tk.MustExec(`set @v1 = 'a', @v2 = 'b', @v3 = 'c'`) - tk.MustQuery(`execute stmt1 using @v1, @v1`).Check(testkit.Rows("a a 1 1 1")) - tk.MustQuery(`execute stmt1 using @v2, @v2`).Check(testkit.Rows("b b 2 2 2")) - tk.MustExec(`prepare stmt2 from "select * from ta, tb where ta.c = tb.b and (ta.a, ta.b) in ((?, ?), (?, ?))"`) - tk.MustQuery(`execute stmt2 using @v1, @v1, @v2, @v2`).Check(testkit.Rows("a a 1 1 1", "b b 2 2 2")) - tk.MustQuery(`execute stmt2 using @v2, @v2, @v3, @v3`).Check(testkit.Rows("b b 2 2 2", "c c 3 3 3")) - - // For issue 19002 - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - tk.MustExec(`drop table if exists t1`) - tk.MustExec(`create table t1(a int, b int, c int, primary key(a, b))`) - tk.MustExec(`insert into t1 values(1,1,111),(2,2,222),(3,3,333)`) - // Point Get: - tk.MustExec(`prepare stmt1 from "select * from t1 where t1.a = ? and t1.b = ?"`) - tk.MustExec(`set @v1=1, @v2=1`) - tk.MustQuery(`execute stmt1 using @v1,@v2`).Check(testkit.Rows("1 1 111")) - tk.MustExec(`set @v1=2, @v2=2`) - tk.MustQuery(`execute stmt1 using @v1,@v2`).Check(testkit.Rows("2 2 222")) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) - // Batch Point Get: - tk.MustExec(`prepare stmt2 from "select * from t1 where (t1.a,t1.b) in ((?,?),(?,?))"`) - tk.MustExec(`set @v1=1, @v2=1, @v3=2, @v4=2`) - tk.MustQuery(`execute stmt2 using @v1,@v2,@v3,@v4`).Check(testkit.Rows("1 1 111", "2 2 222")) - tk.MustExec(`set @v1=2, @v2=2, @v3=3, @v4=3`) - tk.MustQuery(`execute stmt2 using @v1,@v2,@v3,@v4`).Check(testkit.Rows("2 2 222", "3 3 333")) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) -} - -func (s *testPrepareSuite) TestPlanCacheWithDifferentVariableTypes(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - orgEnable := plannercore.PreparedPlanCacheEnabled() - defer func() { - plannercore.SetPreparedPlanCache(orgEnable) - }() - plannercore.SetPreparedPlanCache(true) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("set @@tidb_enable_collect_execution_info=0;") - tk.MustExec("create table t1(a varchar(20), b int, c float, key(b, a))") - tk.MustExec("insert into t1 values('1',1,1.1),('2',2,222),('3',3,333)") - tk.MustExec("create table t2(a varchar(20), b int, c float, key(b, a))") - tk.MustExec("insert into t2 values('3',3,3.3),('2',2,222),('3',3,333)") - - var input []struct { - PrepareStmt string - Executes []struct { - Vars []struct { - Name string - Value string - } - ExecuteSQL string - } - } - var output []struct { - PrepareStmt string - Executes []struct { - SQL string - Vars []struct { - Name string - Value string - } - Plan []string - LastPlanUseCache string - Result []string - } - } - s.testData.GetTestCases(c, &input, &output) - for i, tt := range input { - tk.MustExec(tt.PrepareStmt) - s.testData.OnRecord(func() { - output[i].PrepareStmt = tt.PrepareStmt - output[i].Executes = make([]struct { - SQL string - Vars []struct { - Name string - Value string - } - Plan []string - LastPlanUseCache string - Result []string - }, len(tt.Executes)) - }) - c.Assert(output[i].PrepareStmt, Equals, tt.PrepareStmt) - for j, exec := range tt.Executes { - for _, v := range exec.Vars { - tk.MustExec(fmt.Sprintf(`set @%s = %s`, v.Name, v.Value)) - } - res := tk.MustQuery(exec.ExecuteSQL) - lastPlanUseCache := tk.MustQuery("select @@last_plan_from_cache").Rows()[0][0] - tk.MustQuery(exec.ExecuteSQL) - tkProcess := tk.Se.ShowProcess() - ps := []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - plan := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) - s.testData.OnRecord(func() { - output[i].Executes[j].SQL = exec.ExecuteSQL - output[i].Executes[j].Plan = s.testData.ConvertRowsToStrings(plan.Rows()) - output[i].Executes[j].Vars = exec.Vars - output[i].Executes[j].LastPlanUseCache = lastPlanUseCache.(string) - output[i].Executes[j].Result = s.testData.ConvertRowsToStrings(res.Rows()) - }) - c.Assert(output[i].Executes[j].SQL, Equals, exec.ExecuteSQL) - plan.Check(testkit.Rows(output[i].Executes[j].Plan...)) - c.Assert(output[i].Executes[j].Vars, DeepEquals, exec.Vars) - c.Assert(output[i].Executes[j].LastPlanUseCache, Equals, lastPlanUseCache.(string)) - res.Check(testkit.Rows(output[i].Executes[j].Result...)) - } - } -} - -func (s *testPrepareSuite) TestPlanCacheOperators(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - orgEnable := plannercore.PreparedPlanCacheEnabled() - defer func() { - plannercore.SetPreparedPlanCache(orgEnable) - }() - plannercore.SetPreparedPlanCache(true) - - type ExecCase struct { - Parameters []string - UseCache bool - } - type PrepCase struct { - PrepStmt string - ExecCases []ExecCase - } - - cases := []PrepCase{ - {"use test", nil}, - - // cases for TableReader on PK - {"create table t (a int, b int, primary key(a))", nil}, - {"insert into t values (1,1), (2,2), (3,3), (4,4), (5,5), (6,null)", nil}, - {"select a from t where a=?", []ExecCase{ - {[]string{"1"}, false}, - {[]string{"2"}, true}, - {[]string{"3"}, true}, - }}, - {"select a from t where a in (?,?,?)", []ExecCase{ - {[]string{"1", "1", "1"}, false}, - {[]string{"2", "3", "4"}, true}, - {[]string{"3", "5", "7"}, true}, - }}, - {"select a from t where a>? and a? and a? and a? and a? and a?", []ExecCase{ - {[]string{"1"}, false}, - {[]string{"3"}, true}, - {[]string{"5"}, true}, - }}, - {"select /*+ HASH_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a and t2.b>?", []ExecCase{ - {[]string{"1"}, false}, - {[]string{"3"}, true}, - {[]string{"5"}, true}, - }}, - {"select /*+ HASH_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a and t1.b>? and t2.b?", []ExecCase{ - {[]string{"1"}, false}, - {[]string{"3"}, true}, - {[]string{"5"}, true}, - }}, - {"select /*+ MERGE_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a and t2.b>?", []ExecCase{ - {[]string{"1"}, false}, - {[]string{"3"}, true}, - {[]string{"5"}, true}, - }}, - {"select /*+ MERGE_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a and t1.b>? and t2.b?", []ExecCase{ - {[]string{"1"}, false}, - {[]string{"3"}, true}, - {[]string{"5"}, true}, - }}, - {"select /*+ INL_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a and t2.b>?", []ExecCase{ - {[]string{"1"}, false}, - {[]string{"3"}, true}, - {[]string{"5"}, true}, - }}, - {"select /*+ INL_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a and t1.b>? and t2.b? and t1.a > (select min(t2.a) from t t2 where t2.b < t1.b)", []ExecCase{ - {[]string{"1"}, false}, - {[]string{"3"}, false}, // plans with sub-queries cannot be cached, but the result must be correct - {[]string{"5"}, false}, - }}, - {"select * from t t1 where t1.a > (select min(t2.a) from t t2 where t2.b < t1.b+?)", []ExecCase{ - {[]string{"1"}, false}, - {[]string{"3"}, false}, - {[]string{"5"}, false}, - }}, - {"select * from t t1 where t1.b>? and t1.a > (select min(t2.a) from t t2 where t2.b < t1.b+?)", []ExecCase{ - {[]string{"1", "1"}, false}, - {[]string{"3", "2"}, false}, - {[]string{"5", "3"}, false}, - }}, - {"drop table t", nil}, - - // cases for Window - {"create table t (name varchar(50), y int, sale decimal(14,2))", nil}, - {"insert into t values ('Bob',2016,2.4), ('Bob',2017,3.2), ('Bob',2018,2.1), ('Alice',2016,1.4), ('Alice',2017,2), ('Alice',2018,3.3), ('John',2016,4), ('John',2017,2.1), ('John',2018,5)", nil}, - {"select *, sum(sale) over (partition by y order by sale) total from t where sale>? order by y", []ExecCase{ - {[]string{"0.1"}, false}, - {[]string{"0.5"}, true}, - {[]string{"1.5"}, true}, - {[]string{"3.5"}, true}, - }}, - {"select *, sum(sale) over (partition by y order by sale+? rows 2 preceding) total from t order by y", []ExecCase{ - {[]string{"0.1"}, false}, - {[]string{"0.5"}, true}, - {[]string{"1.5"}, true}, - {[]string{"3.5"}, true}, - }}, - {"select *, rank() over (partition by y order by sale+? rows 2 preceding) total from t order by y", []ExecCase{ - {[]string{"0.1"}, false}, - {[]string{"0.5"}, true}, - {[]string{"1.5"}, true}, - {[]string{"3.5"}, true}, - }}, - {"select *, first_value(sale) over (partition by y order by sale+? rows 2 preceding) total from t order by y", []ExecCase{ - {[]string{"0.1"}, false}, - {[]string{"0.5"}, true}, - {[]string{"1.5"}, true}, - {[]string{"3.5"}, true}, - }}, - {"select *, first_value(sale) over (partition by y order by sale rows ? preceding) total from t order by y", []ExecCase{ - {[]string{"1"}, false}, // window plans with parameters in frame cannot be cached - {[]string{"2"}, false}, - {[]string{"3"}, false}, - {[]string{"4"}, false}, - }}, - {"drop table t", nil}, - - // cases for Limit - {"create table t (a int)", nil}, - {"insert into t values (1), (1), (2), (2), (3), (4), (5), (6), (7), (8), (9), (0), (0)", nil}, - {"select * from t limit ?", []ExecCase{ - {[]string{"20"}, false}, - {[]string{"30"}, false}, - }}, - {"select * from t limit 40, ?", []ExecCase{ - {[]string{"1"}, false}, - {[]string{"2"}, false}, - }}, - {"select * from t limit ?, 10", []ExecCase{ - {[]string{"20"}, false}, - {[]string{"30"}, false}, - }}, - {"select * from t limit ?, ?", []ExecCase{ - {[]string{"20", "20"}, false}, - {[]string{"20", "40"}, false}, - }}, - {"select * from t where a? order by mod(a, 3)", []ExecCase{ - {[]string{"1"}, false}, - {[]string{"2"}, true}, - {[]string{"3"}, true}, - }}, - - // cases for topN - {"select * from t order by b limit ?", []ExecCase{ - {[]string{"1"}, false}, - {[]string{"2"}, false}, - }}, - {"select * from t order by b limit 10, ?", []ExecCase{ - {[]string{"1"}, false}, - {[]string{"2"}, false}, - }}, - {"select * from t order by ? limit 10", []ExecCase{ - {[]string{"1"}, false}, - {[]string{"2"}, false}, - }}, - {"select * from t order by ? limit ?", []ExecCase{ - {[]string{"1", "10"}, false}, - {[]string{"2", "20"}, false}, - }}, - } - - for _, prepCase := range cases { - isQuery := strings.Contains(prepCase.PrepStmt, "select") - if !isQuery { - tk.MustExec(prepCase.PrepStmt) - continue - } - - tk.MustExec(fmt.Sprintf(`prepare stmt from '%v'`, prepCase.PrepStmt)) - for _, execCase := range prepCase.ExecCases { - // set all parameters - usingStmt := "" - if len(execCase.Parameters) > 0 { - setStmt := "set " - usingStmt = "using " - for i, parameter := range execCase.Parameters { - if i > 0 { - setStmt += ", " - usingStmt += ", " - } - setStmt += fmt.Sprintf("@x%v=%v", i, parameter) - usingStmt += fmt.Sprintf("@x%v", i) - } - tk.MustExec(setStmt) - } - - // execute this statement and check whether it uses a cached plan - results := tk.MustQuery("execute stmt " + usingStmt).Sort().Rows() - useCache := "0" - if execCase.UseCache { - useCache = "1" - } - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows(useCache)) - - // check whether the result is correct - tmp := strings.Split(prepCase.PrepStmt, "?") - c.Assert(len(tmp), Equals, len(execCase.Parameters)+1) - query := "" - for i := range tmp { - query += tmp[i] - if i < len(execCase.Parameters) { - query += execCase.Parameters[i] - } - } - tk.MustQuery(query).Sort().Check(results) - } - } + require.Equal(t, int32(1), atomic.LoadInt32(&sm.killed)) } -func (s *testSerialSuite) TestIssue28782(c *C) { +func TestIssue29850(t *testing.T) { store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) + require.NoError(t, err) defer func() { + require.NoError(t, store.Close()) dom.Close() - store.Close() - }() - orgEnable := plannercore.PreparedPlanCacheEnabled() - defer func() { - plannercore.SetPreparedPlanCache(orgEnable) }() - plannercore.SetPreparedPlanCache(true) - tk.MustExec("use test") - tk.MustExec("set @@tidb_enable_collect_execution_info=0;") - tk.MustExec("prepare stmt from 'SELECT IF(?, 1, 0);';") - tk.MustExec("set @a=1, @b=null, @c=0") - tk.MustQuery("execute stmt using @a;").Check(testkit.Rows("1")) - tk.MustQuery("execute stmt using @b;").Check(testkit.Rows("0")) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) - tk.MustQuery("execute stmt using @c;").Check(testkit.Rows("0")) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) -} - -func (s *testSerialSuite) TestIssue29101(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() orgEnable := plannercore.PreparedPlanCacheEnabled() defer func() { plannercore.SetPreparedPlanCache(orgEnable) }() plannercore.SetPreparedPlanCache(true) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`set tidb_enable_clustered_index=on`) + tk.MustExec("set @@tidb_enable_collect_execution_info=0") tk.MustExec(`use test`) - tk.MustExec("set @@tidb_enable_collect_execution_info=0;") tk.MustExec(`CREATE TABLE customer ( c_id int(11) NOT NULL, c_d_id int(11) NOT NULL, - c_w_id int(11) NOT NULL, c_first varchar(16) DEFAULT NULL, + c_w_id int(11) NOT NULL, c_last varchar(16) DEFAULT NULL, c_credit char(2) DEFAULT NULL, c_discount decimal(4,4) DEFAULT NULL, PRIMARY KEY (c_w_id,c_d_id,c_id), - KEY idx_customer (c_w_id,c_d_id,c_last,c_first) - )`) + KEY idx_customer (c_w_id,c_d_id,c_last,c_first))`) tk.MustExec(`CREATE TABLE warehouse ( w_id int(11) NOT NULL, w_tax decimal(4,4) DEFAULT NULL, - PRIMARY KEY (w_id) - )`) - tk.MustExec(`prepare s1 from 'SELECT /*+ TIDB_INLJ(customer,warehouse) */ c_discount, c_last, c_credit, w_tax FROM customer, warehouse WHERE w_id = ? AND c_w_id = w_id AND c_d_id = ? AND c_id = ?'`) - tk.MustExec(`set @a=936,@b=7,@c=158`) - tk.MustQuery(`execute s1 using @a,@b,@c`).Check(testkit.Rows()) - tkProcess := tk.Se.ShowProcess() + PRIMARY KEY (w_id))`) + tk.MustExec(`prepare stmt from 'SELECT c_discount, c_last, c_credit, w_tax + FROM customer, warehouse + WHERE w_id = ? AND c_w_id = w_id AND c_d_id = ? AND c_id = ?'`) + tk.MustExec(`set @w_id=1262`) + tk.MustExec(`set @c_d_id=7`) + tk.MustExec(`set @c_id=1549`) + tk.MustQuery(`execute stmt using @w_id, @c_d_id, @c_id`).Check(testkit.Rows()) + tkProcess := tk.Session().ShowProcess() ps := []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // can use IndexJoin - `Projection_6 1.00 root test.customer.c_discount, test.customer.c_last, test.customer.c_credit, test.warehouse.w_tax`, - `└─IndexJoin_14 1.00 root inner join, inner:TableReader_10, outer key:test.customer.c_w_id, inner key:test.warehouse.w_id, equal cond:eq(test.customer.c_w_id, test.warehouse.w_id)`, - ` ├─Selection_36(Build) 1.00 root eq(test.customer.c_d_id, 7), eq(test.customer.c_id, 158), eq(test.customer.c_w_id, 936)`, - ` │ └─IndexLookUp_35 1.00 root `, - ` │ ├─IndexRangeScan_33(Build) 1.00 cop[tikv] table:customer, index:PRIMARY(c_w_id, c_d_id, c_id) range:[936 7 158,936 7 158], keep order:false, stats:pseudo`, - ` │ └─TableRowIDScan_34(Probe) 1.00 cop[tikv] table:customer keep order:false, stats:pseudo`, - ` └─TableReader_10(Probe) 0.00 root data:Selection_9`, - ` └─Selection_9 0.00 cop[tikv] eq(test.warehouse.w_id, 936)`, - ` └─TableRangeScan_8 1.00 cop[tikv] table:warehouse range: decided by [test.customer.c_w_id], keep order:false, stats:pseudo`)) - tk.MustQuery(`execute s1 using @a,@b,@c`).Check(testkit.Rows()) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) // can use the plan-cache - - tk.MustExec(`CREATE TABLE order_line ( - ol_o_id int(11) NOT NULL, - ol_d_id int(11) NOT NULL, - ol_w_id int(11) NOT NULL, - ol_number int(11) NOT NULL, - ol_i_id int(11) NOT NULL, - PRIMARY KEY (ol_w_id,ol_d_id,ol_o_id,ol_number))`) - tk.MustExec(`CREATE TABLE stock ( - s_i_id int(11) NOT NULL, - s_w_id int(11) NOT NULL, - s_quantity int(11) DEFAULT NULL, - PRIMARY KEY (s_w_id,s_i_id))`) - tk.MustExec(`prepare s1 from 'SELECT /*+ TIDB_INLJ(order_line,stock) */ COUNT(DISTINCT (s_i_id)) stock_count FROM order_line, stock WHERE ol_w_id = ? AND ol_d_id = ? AND ol_o_id < ? AND ol_o_id >= ? - 20 AND s_w_id = ? AND s_i_id = ol_i_id AND s_quantity < ?'`) - tk.MustExec(`set @a=391,@b=1,@c=3058,@d=18`) - tk.MustExec(`execute s1 using @a,@b,@c,@c,@a,@d`) - tkProcess = tk.Se.ShowProcess() + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // can use PointGet + `Projection_7 0.00 root test.customer.c_discount, test.customer.c_last, test.customer.c_credit, test.warehouse.w_tax`, + `└─MergeJoin_8 0.00 root inner join, left key:test.customer.c_w_id, right key:test.warehouse.w_id`, + ` ├─Point_Get_34(Build) 1.00 root table:warehouse handle:1262`, + ` └─Point_Get_33(Probe) 1.00 root table:customer, clustered index:PRIMARY(c_w_id, c_d_id, c_id) `)) + tk.MustQuery(`execute stmt using @w_id, @c_d_id, @c_id`).Check(testkit.Rows()) + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) // can use the cached plan + + tk.MustExec(`create table t (a int primary key)`) + tk.MustExec(`insert into t values (1), (2)`) + tk.MustExec(`prepare stmt from 'select * from t where a>=? and a<=?'`) + tk.MustExec(`set @a1=1, @a2=2`) + tk.MustQuery(`execute stmt using @a1, @a1`).Check(testkit.Rows("1")) + tkProcess = tk.Session().ShowProcess() ps = []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // can use index-join - `StreamAgg_9 1.00 root funcs:count(distinct test.stock.s_i_id)->Column#11`, - `└─IndexJoin_14 0.03 root inner join, inner:IndexLookUp_13, outer key:test.order_line.ol_i_id, inner key:test.stock.s_i_id, equal cond:eq(test.order_line.ol_i_id, test.stock.s_i_id)`, - ` ├─Selection_30(Build) 0.03 root eq(test.order_line.ol_d_id, 1), eq(test.order_line.ol_w_id, 391), ge(test.order_line.ol_o_id, 3038), lt(test.order_line.ol_o_id, 3058)`, - ` │ └─IndexLookUp_29 0.03 root `, - ` │ ├─IndexRangeScan_27(Build) 0.03 cop[tikv] table:order_line, index:PRIMARY(ol_w_id, ol_d_id, ol_o_id, ol_number) range:[391 1 3038,391 1 3058), keep order:false, stats:pseudo`, - ` │ └─TableRowIDScan_28(Probe) 0.03 cop[tikv] table:order_line keep order:false, stats:pseudo`, - ` └─IndexLookUp_13(Probe) 1.00 root `, - ` ├─IndexRangeScan_10(Build) 1.00 cop[tikv] table:stock, index:PRIMARY(s_w_id, s_i_id) range: decided by [eq(test.stock.s_i_id, test.order_line.ol_i_id) eq(test.stock.s_w_id, 391)], keep order:false, stats:pseudo`, - ` └─Selection_12(Probe) 1.00 cop[tikv] lt(test.stock.s_quantity, 18)`, - ` └─TableRowIDScan_11 1.00 cop[tikv] table:stock keep order:false, stats:pseudo`)) - tk.MustExec(`execute s1 using @a,@b,@c,@c,@a,@d`) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) // can use the plan-cache -} - -func (s *testSerialSuite) TestIssue28087And28162(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - orgEnable := plannercore.PreparedPlanCacheEnabled() - defer func() { - plannercore.SetPreparedPlanCache(orgEnable) - }() - plannercore.SetPreparedPlanCache(true) - - // issue 28087 - tk.MustExec(`use test`) - tk.MustExec(`drop table if exists IDT_26207`) - tk.MustExec(`CREATE TABLE IDT_26207 (col1 bit(1))`) - tk.MustExec(`insert into IDT_26207 values(0x0), (0x1)`) - tk.MustExec(`prepare stmt from 'select t1.col1 from IDT_26207 as t1 left join IDT_26207 as t2 on t1.col1 = t2.col1 where t1.col1 in (?, ?, ?)'`) - tk.MustExec(`set @a=0x01, @b=0x01, @c=0x01`) - tk.MustQuery(`execute stmt using @a,@b,@c`).Check(testkit.Rows("\x01")) - tk.MustExec(`set @a=0x00, @b=0x00, @c=0x01`) - tk.MustQuery(`execute stmt using @a,@b,@c`).Check(testkit.Rows("\x00", "\x01")) - tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) - - // issue 28162 - tk.MustExec(`drop table if exists IDT_MC21780`) - tk.MustExec(`CREATE TABLE IDT_MC21780 ( - COL1 timestamp NULL DEFAULT NULL, - COL2 timestamp NULL DEFAULT NULL, - COL3 timestamp NULL DEFAULT NULL, - KEY U_M_COL (COL1,COL2) - )`) - tk.MustExec(`insert into IDT_MC21780 values("1970-12-18 10:53:28", "1970-12-18 10:53:28", "1970-12-18 10:53:28")`) - tk.MustExec(`prepare stmt from 'select/*+ hash_join(t1) */ * from IDT_MC21780 t1 join IDT_MC21780 t2 on t1.col1 = t2.col1 where t1. col1 < ? and t2. col1 in (?, ?, ?);'`) - tk.MustExec(`set @a="2038-01-19 03:14:07", @b="2038-01-19 03:14:07", @c="2038-01-19 03:14:07", @d="2038-01-19 03:14:07"`) - tk.MustQuery(`execute stmt using @a,@b,@c,@d`).Check(testkit.Rows()) - tk.MustExec(`set @a="1976-09-09 20:21:11", @b="2021-07-14 09:28:16", @c="1982-01-09 03:36:39", @d="1970-12-18 10:53:28"`) - tk.MustQuery(`execute stmt using @a,@b,@c,@d`).Check(testkit.Rows("1970-12-18 10:53:28 1970-12-18 10:53:28 1970-12-18 10:53:28 1970-12-18 10:53:28 1970-12-18 10:53:28 1970-12-18 10:53:28")) + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // cannot use PointGet since it contains a range condition + `Selection_7 1.00 root ge(test.t.a, 1), le(test.t.a, 1)`, + `└─TableReader_6 1.00 root data:TableRangeScan_5`, + ` └─TableRangeScan_5 1.00 cop[tikv] table:t range:[1,1], keep order:false, stats:pseudo`)) + tk.MustQuery(`execute stmt using @a1, @a2`).Check(testkit.Rows("1", "2")) tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) -} - -func (s *testPrepareSuite) TestParameterPushDown(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - orgEnable := plannercore.PreparedPlanCacheEnabled() - defer func() { - plannercore.SetPreparedPlanCache(orgEnable) - }() - plannercore.SetPreparedPlanCache(true) - - tk.MustExec(`use test`) - tk.MustExec(`drop table if exists t`) - tk.MustExec(`create table t (a int, b int, c int, key(a))`) - tk.MustExec(`insert into t values (1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5), (6, 6, 6)`) - tk.MustExec("set @@tidb_enable_collect_execution_info=0;") - tk.MustExec(`set @x1=1,@x5=5,@x10=10,@x20=20`) - - var input []struct { - SQL string - } - var output []struct { - Result []string - Plan []string - FromCache string - } - s.testData.GetTestCases(c, &input, &output) - - for i, tt := range input { - if strings.HasPrefix(tt.SQL, "execute") { - res := tk.MustQuery(tt.SQL).Sort() - fromCache := tk.MustQuery("select @@last_plan_from_cache") - tk.MustQuery(tt.SQL) - tkProcess := tk.Se.ShowProcess() - ps := []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - plan := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) - - s.testData.OnRecord(func() { - output[i].Result = s.testData.ConvertRowsToStrings(res.Rows()) - output[i].Plan = s.testData.ConvertRowsToStrings(plan.Rows()) - output[i].FromCache = fromCache.Rows()[0][0].(string) - }) - - res.Check(testkit.Rows(output[i].Result...)) - plan.Check(testkit.Rows(output[i].Plan...)) - c.Assert(output[i].FromCache, Equals, fromCache.Rows()[0][0].(string)) - } else { - tk.MustExec(tt.SQL) - s.testData.OnRecord(func() { - output[i].Result = nil - }) - } - } -} - -func (s *testSerialSuite) TestPreparePlanCache4Function(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - orgEnable := plannercore.PreparedPlanCacheEnabled() - defer func() { - plannercore.SetPreparedPlanCache(orgEnable) - }() - plannercore.SetPreparedPlanCache(true) - tk.MustExec("use test") - tk.MustExec("set @@tidb_enable_collect_execution_info=0;") - - // Testing for non-deterministic functions - tk.MustExec("prepare stmt from 'select rand()';") - res := tk.MustQuery("execute stmt;") - c.Assert(len(res.Rows()), Equals, 1) - - res1 := tk.MustQuery("execute stmt;") - c.Assert(len(res1.Rows()), Equals, 1) - c.Assert(res.Rows()[0][0] != res1.Rows()[0][0], Equals, true) - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) - - // Testing for control functions - tk.MustExec("prepare stmt from 'SELECT IFNULL(?,0);';") - tk.MustExec("set @a = 1, @b = null;") - tk.MustQuery("execute stmt using @a;").Check(testkit.Rows("1")) - tk.MustQuery("execute stmt using @b;").Check(testkit.Rows("0")) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) - - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a int);") - tk.MustExec("prepare stmt from 'select a, case when a = ? then 0 when a <=> ? then 1 else 2 end b from t order by a;';") - tk.MustExec("insert into t values(0), (1), (2), (null);") - tk.MustExec("set @a = 0, @b = 1, @c = 2, @d = null;") - tk.MustQuery("execute stmt using @a, @b;").Check(testkit.Rows(" 2", "0 0", "1 1", "2 2")) - tk.MustQuery("execute stmt using @c, @d;").Check(testkit.Rows(" 1", "0 2", "1 2", "2 0")) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) -} - -func (s *testSerialSuite) TestPreparePlanCache4DifferentSystemVars(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - orgEnable := plannercore.PreparedPlanCacheEnabled() - defer func() { - plannercore.SetPreparedPlanCache(orgEnable) - }() - plannercore.SetPreparedPlanCache(true) - tk.MustExec("use test") - tk.MustExec("set @@tidb_enable_collect_execution_info=0;") - - // Testing for 'sql_select_limit' - tk.MustExec("set @@sql_select_limit = 1") - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a int);") - tk.MustExec("insert into t values(0), (1), (null);") - tk.MustExec("prepare stmt from 'select a from t order by a;';") - tk.MustQuery("execute stmt;").Check(testkit.Rows("")) - - tk.MustExec("set @@sql_select_limit = 2") - tk.MustQuery("execute stmt;").Check(testkit.Rows("", "0")) - // The 'sql_select_limit' will be stored in the cache key. So if the `sql_select_limit` - // have been changed, the plan cache can not be reused. - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) - - tk.MustExec("set @@sql_select_limit = 18446744073709551615") - tk.MustQuery("execute stmt;").Check(testkit.Rows("", "0", "1")) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) - - // test for 'tidb_enable_index_merge' - tk.MustExec("set @@tidb_enable_index_merge = 1;") - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a int, b int, index idx_a(a), index idx_b(b));") - tk.MustExec("prepare stmt from 'select * from t use index(idx_a, idx_b) where a > 1 or b > 1;';") - tk.MustExec("execute stmt;") - tkProcess := tk.Se.ShowProcess() - ps := []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - res := tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - c.Assert(len(res.Rows()), Equals, 4) - c.Assert(res.Rows()[0][0], Matches, ".*IndexMerge.*") - - tk.MustExec("set @@tidb_enable_index_merge = 0;") - tk.MustExec("execute stmt;") - tkProcess = tk.Se.ShowProcess() - ps = []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - c.Assert(len(res.Rows()), Equals, 4) - c.Assert(res.Rows()[0][0], Matches, ".*IndexMerge.*") - tk.MustExec("execute stmt;") - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) - - // test for 'tidb_enable_parallel_apply' - tk.MustExec("set @@tidb_enable_collect_execution_info=1;") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a int, b int)") - tk.MustExec("insert into t values (0, 0), (1, 1), (2, 2), (3, 3), (4, 4), (5, 5), (6, 6), (7, 7), (8, 8), (9, 9), (null, null)") - - tk.MustExec("set tidb_enable_parallel_apply=true") - tk.MustExec("prepare stmt from 'select t1.b from t t1 where t1.b > (select max(b) from t t2 where t1.a > t2.a);';") - tk.MustQuery("execute stmt;").Sort().Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7", "8", "9")) - tkProcess = tk.Se.ShowProcess() - ps = []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - c.Assert(res.Rows()[1][0], Matches, ".*Apply.*") - c.Assert(res.Rows()[1][5], Matches, ".*Concurrency.*") - - tk.MustExec("set tidb_enable_parallel_apply=false") - tk.MustQuery("execute stmt;").Sort().Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7", "8", "9")) - tkProcess = tk.Se.ShowProcess() - ps = []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - c.Assert(res.Rows()[1][0], Matches, ".*Apply.*") - executionInfo := fmt.Sprintf("%v", res.Rows()[1][4]) - // Do not use the parallel apply. - c.Assert(strings.Contains(executionInfo, "Concurrency") == false, Equals, true) - tk.MustExec("execute stmt;") - // The subquery plan can not be cached. - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) - // test for apply cache - tk.MustExec("set @@tidb_enable_collect_execution_info=1;") - tk.MustExec("set tidb_mem_quota_apply_cache=33554432") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a int, b int)") - tk.MustExec("insert into t values (0, 0), (1, 1), (2, 2), (3, 3), (4, 4), (5, 5), (6, 6), (7, 7), (8, 8), (9, 9), (null, null)") - - tk.MustExec("prepare stmt from 'select t1.b from t t1 where t1.b > (select max(b) from t t2 where t1.a > t2.a);';") - tk.MustQuery("execute stmt;").Sort().Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7", "8", "9")) - tkProcess = tk.Se.ShowProcess() - ps = []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - c.Assert(res.Rows()[1][0], Matches, ".*Apply.*") - c.Assert(res.Rows()[1][5], Matches, ".*cache:ON.*") - - tk.MustExec("set tidb_mem_quota_apply_cache=0") - tk.MustQuery("execute stmt;").Sort().Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7", "8", "9")) - tkProcess = tk.Se.ShowProcess() + tk.MustExec(`prepare stmt from 'select * from t where a=? or a=?'`) + tk.MustQuery(`execute stmt using @a1, @a1`).Check(testkit.Rows("1")) + tkProcess = tk.Session().ShowProcess() ps = []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - res = tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - c.Assert(res.Rows()[1][0], Matches, ".*Apply.*") - executionInfo = fmt.Sprintf("%v", res.Rows()[1][5]) - // Do not use the apply cache. - c.Assert(strings.Contains(executionInfo, "cache:OFF") == true, Equals, true) - tk.MustExec("execute stmt;") - // The subquery plan can not be cached. - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) -} - -func (s *testSerialSuite) TestTemporaryTable4PlanCache(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - orgEnable := plannercore.PreparedPlanCacheEnabled() - defer func() { - plannercore.SetPreparedPlanCache(orgEnable) - }() - plannercore.SetPreparedPlanCache(true) - tk.MustExec("use test") - tk.MustExec("set @@tidb_enable_collect_execution_info=0;") - tk.MustExec("drop table if exists tmp2") - tk.MustExec("create temporary table tmp2 (a int, b int, key(a), key(b));") - tk.MustExec("prepare stmt from 'select * from tmp2;';") - tk.MustQuery("execute stmt;").Check(testkit.Rows()) - tk.MustQuery("execute stmt;").Check(testkit.Rows()) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) - - tk.MustExec("drop table if exists tmp_t;") - tk.MustExec("create global temporary table tmp_t (id int primary key, a int, b int, index(a)) on commit delete rows") - tk.MustExec("prepare stmt from 'select * from tmp_t;';") - tk.MustQuery("execute stmt;").Check(testkit.Rows()) - tk.MustQuery("execute stmt;").Check(testkit.Rows()) - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) - -} - -func (s *testSerialSuite) TestPreparePlanCache4Blacklist(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - orgEnable := plannercore.PreparedPlanCacheEnabled() - defer func() { - plannercore.SetPreparedPlanCache(orgEnable) - }() - plannercore.SetPreparedPlanCache(true) - tk.MustExec("use test") - tk.MustExec("set @@tidb_enable_collect_execution_info=0;") - - // test the blacklist of optimization rules - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a int);") - tk.MustExec("prepare stmt from 'select min(a) from t;';") - tk.MustExec("execute stmt;") - tkProcess := tk.Se.ShowProcess() - ps := []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - res := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) - c.Assert(res.Rows()[1][0], Matches, ".*TopN.*") - - res = tk.MustQuery("explain format = 'brief' select min(a) from t") - c.Assert(res.Rows()[1][0], Matches, ".*TopN.*") - - tk.MustExec("INSERT INTO mysql.opt_rule_blacklist VALUES('max_min_eliminate');") - tk.MustExec("ADMIN reload opt_rule_blacklist;") - - tk.MustExec("execute stmt;") - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) - tk.MustExec("execute stmt;") - tkProcess = tk.Se.ShowProcess() - ps = []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - res = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) - // Plans that have been cached will not be affected by the blacklist. - c.Assert(res.Rows()[1][0], Matches, ".*TopN.*") - - res = tk.MustQuery("explain format = 'brief' select min(a) from t") - c.Assert(res.Rows()[0][0], Matches, ".*StreamAgg.*") - - // test the blacklist of Expression Pushdown - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a int);") - tk.MustExec("prepare stmt from 'SELECT * FROM t WHERE a < 2 and a > 2;';") - tk.MustExec("execute stmt;") - tkProcess = tk.Se.ShowProcess() - ps = []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - res = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) - c.Assert(len(res.Rows()), Equals, 3) - c.Assert(res.Rows()[1][0], Matches, ".*Selection.*") - c.Assert(res.Rows()[1][4], Equals, "gt(test.t.a, 2), lt(test.t.a, 2)") - - res = tk.MustQuery("explain format = 'brief' SELECT * FROM t WHERE a < 2 and a > 2;") - c.Assert(len(res.Rows()), Equals, 3) - c.Assert(res.Rows()[1][4], Equals, "gt(test.t.a, 2), lt(test.t.a, 2)") - - tk.MustExec("INSERT INTO mysql.expr_pushdown_blacklist VALUES('<','tikv','');") - tk.MustExec("ADMIN reload expr_pushdown_blacklist;") - - tk.MustExec("execute stmt;") - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) - tk.MustExec("execute stmt;") - tkProcess = tk.Se.ShowProcess() - ps = []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - res = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) - // The expressions can still be pushed down to tikv. - c.Assert(len(res.Rows()), Equals, 3) - c.Assert(res.Rows()[1][0], Matches, ".*Selection.*") - c.Assert(res.Rows()[1][4], Equals, "gt(test.t.a, 2), lt(test.t.a, 2)") - - res = tk.MustQuery("explain format = 'brief' SELECT * FROM t WHERE a < 2 and a > 2;") - c.Assert(len(res.Rows()), Equals, 4) - c.Assert(res.Rows()[0][0], Matches, ".*Selection.*") - c.Assert(res.Rows()[0][4], Equals, "lt(test.t.a, 2)") - c.Assert(res.Rows()[2][0], Matches, ".*Selection.*") - c.Assert(res.Rows()[2][4], Equals, "gt(test.t.a, 2)") - - tk.MustExec("DELETE FROM mysql.expr_pushdown_blacklist;") - tk.MustExec("ADMIN reload expr_pushdown_blacklist;") -} - -func (s *testSerialSuite) TestIssue28064(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - orgEnable := plannercore.PreparedPlanCacheEnabled() - defer func() { - plannercore.SetPreparedPlanCache(orgEnable) - }() - plannercore.SetPreparedPlanCache(true) - tk.MustExec("use test") - tk.MustExec("drop table if exists t28064") - tk.MustExec("CREATE TABLE `t28064` (" + - "`a` decimal(10,0) DEFAULT NULL," + - "`b` decimal(10,0) DEFAULT NULL," + - "`c` decimal(10,0) DEFAULT NULL," + - "`d` decimal(10,0) DEFAULT NULL," + - "KEY `iabc` (`a`,`b`,`c`));") - tk.MustExec("set @a='123', @b='234', @c='345';") - tk.MustExec("set @@tidb_enable_collect_execution_info=0;") - tk.MustExec("prepare stmt1 from 'select * from t28064 use index (iabc) where a = ? and b = ? and c = ?';") - - tk.MustExec("execute stmt1 using @a, @b, @c;") - tkProcess := tk.Se.ShowProcess() - ps := []*util.ProcessInfo{tkProcess} - tk.Se.SetSessionManager(&mockSessionManager1{PS: ps}) - rows := tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) - rows.Check(testkit.Rows("Selection_8 0.00 root eq(test.t28064.a, 123), eq(test.t28064.b, 234), eq(test.t28064.c, 345)", - "└─IndexLookUp_7 0.00 root ", - " ├─IndexRangeScan_5(Build) 0.00 cop[tikv] table:t28064, index:iabc(a, b, c) range:[123 234 345,123 234 345], keep order:false, stats:pseudo", - " └─TableRowIDScan_6(Probe) 0.00 cop[tikv] table:t28064 keep order:false, stats:pseudo")) - - tk.MustExec("execute stmt1 using @a, @b, @c;") - rows = tk.MustQuery("select @@last_plan_from_cache") - rows.Check(testkit.Rows("1")) - - tk.MustExec("execute stmt1 using @a, @b, @c;") - rows = tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)) - rows.Check(testkit.Rows("Selection_8 0.00 root eq(test.t28064.a, 123), eq(test.t28064.b, 234), eq(test.t28064.c, 345)", - "└─IndexLookUp_7 0.00 root ", - " ├─IndexRangeScan_5(Build) 0.00 cop[tikv] table:t28064, index:iabc(a, b, c) range:[123 234 345,123 234 345], keep order:false, stats:pseudo", - " └─TableRowIDScan_6(Probe) 0.00 cop[tikv] table:t28064 keep order:false, stats:pseudo")) + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // cannot use PointGet since it contains a or condition + `Selection_7 1.00 root or(eq(test.t.a, 1), eq(test.t.a, 1))`, + `└─TableReader_6 1.00 root data:TableRangeScan_5`, + ` └─TableRangeScan_5 1.00 cop[tikv] table:t range:[1,1], keep order:false, stats:pseudo`)) + tk.MustQuery(`execute stmt using @a1, @a2`).Check(testkit.Rows("1", "2")) } diff --git a/executor/replace.go b/executor/replace.go index cd7a3d44a7bb4..78e0085aa520e 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -23,12 +23,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "go.uber.org/zap" @@ -99,11 +99,7 @@ func (e *ReplaceExec) EqualDatumsAsBinary(sc *stmtctx.StatementContext, a []type return false, nil } for i, ai := range a { - collation := ai.Collation() - // We should use binary collation to compare datum, otherwise the result will be incorrect - ai.SetCollation(charset.CollationBin) - v, err := ai.CompareDatum(sc, &b[i]) - ai.SetCollation(collation) + v, err := ai.Compare(sc, &b[i], collate.GetBinaryCollator()) if err != nil { return false, errors.Trace(err) } @@ -226,7 +222,7 @@ func (e *ReplaceExec) exec(ctx context.Context, newRows [][]types.Datum) error { defer snapshot.SetOption(kv.CollectRuntimeStats, nil) } } - setResourceGroupTagForTxn(e.ctx.GetSessionVars().StmtCtx, txn) + setResourceGroupTaggerForTxn(e.ctx.GetSessionVars().StmtCtx, txn) prefetchStart := time.Now() // Use BatchGet to fill cache. // It's an optimization and could be removed without affecting correctness. diff --git a/executor/show.go b/executor/show.go index 9fcbb67e7cf66..86ace8956dfda 100644 --- a/executor/show.go +++ b/executor/show.go @@ -1193,9 +1193,12 @@ func (e *ShowExec) fetchShowCreateView() error { func fetchShowCreateTable4View(ctx sessionctx.Context, tb *model.TableInfo, buf *bytes.Buffer) { sqlMode := ctx.GetSessionVars().SQLMode - fmt.Fprintf(buf, "CREATE ALGORITHM=%s ", tb.View.Algorithm.String()) - fmt.Fprintf(buf, "DEFINER=%s@%s ", stringutil.Escape(tb.View.Definer.Username, sqlMode), stringutil.Escape(tb.View.Definer.Hostname, sqlMode)) + if tb.View.Definer.AuthUsername == "" || tb.View.Definer.AuthHostname == "" { + fmt.Fprintf(buf, "DEFINER=%s@%s ", stringutil.Escape(tb.View.Definer.Username, sqlMode), stringutil.Escape(tb.View.Definer.Hostname, sqlMode)) + } else { + fmt.Fprintf(buf, "DEFINER=%s@%s ", stringutil.Escape(tb.View.Definer.AuthUsername, sqlMode), stringutil.Escape(tb.View.Definer.AuthHostname, sqlMode)) + } fmt.Fprintf(buf, "SQL SECURITY %s ", tb.View.Security.String()) fmt.Fprintf(buf, "VIEW %s (", stringutil.Escape(tb.Name.O, sqlMode)) for i, col := range tb.Columns { diff --git a/executor/show_placement.go b/executor/show_placement.go index 327157bfe74bd..d77c0a31000f1 100644 --- a/executor/show_placement.go +++ b/executor/show_placement.go @@ -202,8 +202,8 @@ func (e *ShowExec) fetchShowPlacementForPartition(ctx context.Context) (err erro } var partition *model.PartitionDefinition - for _, par := range tblInfo.Partition.Definitions { - par := par + for i := range tblInfo.Partition.Definitions { + par := tblInfo.Partition.Definitions[i] if par.Name.L == e.Partition.L { partition = &par break @@ -330,7 +330,8 @@ func (e *ShowExec) fetchAllTablePlacements(ctx context.Context, scheduleState ma } if tblInfo.Partition != nil { - for _, partition := range tblInfo.Partition.Definitions { + for i := range tblInfo.Partition.Definitions { + partition := tblInfo.Partition.Definitions[i] partitionPlacement, err := e.getPartitionPlacement(tblPlacement, &partition) if err != nil { return err diff --git a/executor/show_stats.go b/executor/show_stats.go index 9ea89810a649e..b3a6a5f809b82 100644 --- a/executor/show_stats.go +++ b/executor/show_stats.go @@ -446,8 +446,8 @@ func (e *ShowExec) appendTableForStatsHealthy(dbName, tblName, partitionName str func (e *ShowExec) fetchShowAnalyzeStatus() { rows := dataForAnalyzeStatusHelper(e.baseExecutor.ctx) for _, row := range rows { - for i, val := range row { - e.result.AppendDatum(i, &val) + for i := range row { + e.result.AppendDatum(i, &row[i]) } } } @@ -498,14 +498,10 @@ func (e *ShowExec) fetchShowColumnStatsUsage() error { pi := tbl.GetPartitionInfo() if pi == nil || e.ctx.GetSessionVars().UseDynamicPartitionPrune() { appendTableForColumnStatsUsage(db.Name.O, tbl, pi != nil, nil) - if pi != nil { - for _, def := range pi.Definitions { - appendTableForColumnStatsUsage(db.Name.O, tbl, false, &def) - } - } - } else { - for _, def := range pi.Definitions { - appendTableForColumnStatsUsage(db.Name.O, tbl, false, &def) + } + if pi != nil { + for i := range pi.Definitions { + appendTableForColumnStatsUsage(db.Name.O, tbl, false, &pi.Definitions[i]) } } } diff --git a/executor/show_stats_serial_test.go b/executor/show_stats_serial_test.go new file mode 100644 index 0000000000000..c2f7e7ca828ee --- /dev/null +++ b/executor/show_stats_serial_test.go @@ -0,0 +1,73 @@ +// 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 ( + "testing" + + "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" +) + +func TestShowAnalyzeStatus(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + statistics.ClearHistoryJobs() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int, primary key(a), index idx(b))") + tk.MustExec(`insert into t values (1, 1), (2, 2)`) + + tk.MustExec("set @@tidb_analyze_version=2") + tk.MustExec("analyze table t") + result := tk.MustQuery("show analyze status").Sort() + require.Len(t, result.Rows(), 1) + require.Equal(t, "test", result.Rows()[0][0]) + require.Equal(t, "t", result.Rows()[0][1]) + require.Equal(t, "", result.Rows()[0][2]) + require.Equal(t, "analyze table", result.Rows()[0][3]) + require.Equal(t, "2", result.Rows()[0][4]) + require.NotNil(t, result.Rows()[0][5]) + require.NotNil(t, result.Rows()[0][6]) + require.Equal(t, "finished", result.Rows()[0][7]) + + statistics.ClearHistoryJobs() + + tk.MustExec("set @@tidb_analyze_version=1") + tk.MustExec("analyze table t") + result = tk.MustQuery("show analyze status").Sort() + require.Len(t, result.Rows(), 2) + require.Equal(t, "test", result.Rows()[0][0]) + require.Equal(t, "t", result.Rows()[0][1]) + require.Equal(t, "", result.Rows()[0][2]) + require.Equal(t, "analyze columns", result.Rows()[0][3]) + require.Equal(t, "2", result.Rows()[0][4]) + require.NotNil(t, result.Rows()[0][5]) + require.NotNil(t, result.Rows()[0][6]) + require.Equal(t, "finished", result.Rows()[0][7]) + + require.Len(t, result.Rows(), 2) + require.Equal(t, "test", result.Rows()[1][0]) + require.Equal(t, "t", result.Rows()[1][1]) + require.Equal(t, "", result.Rows()[1][2]) + require.Equal(t, "analyze index idx", result.Rows()[1][3]) + require.Equal(t, "2", result.Rows()[1][4]) + require.NotNil(t, result.Rows()[1][5]) + require.NotNil(t, result.Rows()[1][6]) + require.Equal(t, "finished", result.Rows()[1][7]) +} diff --git a/executor/show_stats_test.go b/executor/show_stats_test.go index 986b69e8b35da..de55cfa595aa4 100644 --- a/executor/show_stats_test.go +++ b/executor/show_stats_test.go @@ -16,66 +16,74 @@ package executor_test import ( "fmt" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/statistics" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) -type testShowStatsSuite struct { - *baseTestSuite -} +func TestShowStatsMeta(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() -func (s *testShowStatsSuite) TestShowStatsMeta(c *C) { - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t, t1") tk.MustExec("create table t (a int, b int)") tk.MustExec("create table t1 (a int, b int)") tk.MustExec("analyze table t, t1") result := tk.MustQuery("show stats_meta") - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][1], Equals, "t") - c.Assert(result.Rows()[1][1], Equals, "t1") + require.Len(t, result.Rows(), 2) + require.Equal(t, "t", result.Rows()[0][1]) + require.Equal(t, "t1", result.Rows()[1][1]) result = tk.MustQuery("show stats_meta where table_name = 't'") - c.Assert(len(result.Rows()), Equals, 1) - c.Assert(result.Rows()[0][1], Equals, "t") + require.Len(t, result.Rows(), 1) + require.Equal(t, "t", result.Rows()[0][1]) } -func (s *testShowStatsSuite) TestShowStatsHistograms(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowStatsHistograms(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 t") tk.MustExec("create table t (a int, b int)") tk.MustExec("analyze table t") result := tk.MustQuery("show stats_histograms") - c.Assert(len(result.Rows()), Equals, 0) + require.Len(t, result.Rows(), 0) tk.MustExec("insert into t values(1,1)") tk.MustExec("analyze table t") result = tk.MustQuery("show stats_histograms").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][3], Equals, "a") - c.Assert(result.Rows()[1][3], Equals, "b") + require.Len(t, result.Rows(), 2) + require.Equal(t, "a", result.Rows()[0][3]) + require.Equal(t, "b", result.Rows()[1][3]) result = tk.MustQuery("show stats_histograms where column_name = 'a'") - c.Assert(len(result.Rows()), Equals, 1) - c.Assert(result.Rows()[0][3], Equals, "a") + require.Len(t, result.Rows(), 1) + require.Equal(t, "a", result.Rows()[0][3]) tk.MustExec("drop table t") tk.MustExec("create table t(a int, b int, c int, index idx_b(b), index idx_c_a(c, a))") tk.MustExec("insert into t values(1,null,1),(2,null,2),(3,3,3),(4,null,4),(null,null,null)") res := tk.MustQuery("show stats_histograms where table_name = 't'") - c.Assert(len(res.Rows()), Equals, 0) + require.Len(t, res.Rows(), 0) tk.MustExec("analyze table t index idx_b") res = tk.MustQuery("show stats_histograms where table_name = 't' and column_name = 'idx_b'") - c.Assert(len(res.Rows()), Equals, 1) + require.Len(t, res.Rows(), 1) } -func (s *testShowStatsSuite) TestShowStatsBuckets(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowStatsBuckets(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 t") // Simple behavior testing. Version=1 is enough. @@ -117,8 +125,12 @@ func (s *testShowStatsSuite) TestShowStatsBuckets(c *C) { result.Check(testkit.Rows("test t idx 1 0 1 1 (2020-01-01 00:00:00, 1) (2020-01-01 00:00:00, 1) 0")) } -func (s *testShowStatsSuite) TestShowStatsHasNullValue(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowStatsHasNullValue(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 t") tk.MustExec("create table t (a int, index idx(a))") @@ -143,44 +155,48 @@ func (s *testShowStatsSuite) TestShowStatsHasNullValue(c *C) { tk.MustExec("create table t(a int, b int, c int, index idx_b(b), index idx_c_a(c, a))") tk.MustExec("insert into t values(1,null,1),(2,null,2),(3,3,3),(4,null,4),(null,null,null)") res := tk.MustQuery("show stats_histograms where table_name = 't'") - c.Assert(len(res.Rows()), Equals, 0) + require.Len(t, res.Rows(), 0) tk.MustExec("analyze table t index idx_b") res = tk.MustQuery("show stats_histograms where table_name = 't' and column_name = 'idx_b'") - c.Assert(len(res.Rows()), Equals, 1) - c.Assert(res.Rows()[0][7], Equals, "4") + require.Len(t, res.Rows(), 1) + require.Equal(t, "4", res.Rows()[0][7]) res = tk.MustQuery("show stats_histograms where table_name = 't' and column_name = 'b'") - c.Assert(len(res.Rows()), Equals, 0) + require.Len(t, res.Rows(), 0) tk.MustExec("analyze table t index idx_c_a") res = tk.MustQuery("show stats_histograms where table_name = 't' and column_name = 'idx_c_a'") - c.Assert(len(res.Rows()), Equals, 1) - c.Assert(res.Rows()[0][7], Equals, "0") + require.Len(t, res.Rows(), 1) + require.Equal(t, "0", res.Rows()[0][7]) res = tk.MustQuery("show stats_histograms where table_name = 't' and column_name = 'c'") - c.Assert(len(res.Rows()), Equals, 0) + require.Len(t, res.Rows(), 0) res = tk.MustQuery("show stats_histograms where table_name = 't' and column_name = 'a'") - c.Assert(len(res.Rows()), Equals, 0) + require.Len(t, res.Rows(), 0) tk.MustExec("truncate table t") tk.MustExec("insert into t values(1,null,1),(2,null,2),(3,3,3),(4,null,4),(null,null,null)") res = tk.MustQuery("show stats_histograms where table_name = 't'") - c.Assert(len(res.Rows()), Equals, 0) + require.Len(t, res.Rows(), 0) tk.MustExec("analyze table t index") res = tk.MustQuery("show stats_histograms where table_name = 't'").Sort() - c.Assert(len(res.Rows()), Equals, 2) - c.Assert(res.Rows()[0][7], Equals, "4") - c.Assert(res.Rows()[1][7], Equals, "0") + require.Len(t, res.Rows(), 2) + require.Equal(t, "4", res.Rows()[0][7]) + require.Equal(t, "0", res.Rows()[1][7]) tk.MustExec("truncate table t") tk.MustExec("insert into t values(1,null,1),(2,null,2),(3,3,3),(4,null,4),(null,null,null)") tk.MustExec("analyze table t") res = tk.MustQuery("show stats_histograms where table_name = 't'").Sort() - c.Assert(len(res.Rows()), Equals, 5) - c.Assert(res.Rows()[0][7], Equals, "1") - c.Assert(res.Rows()[1][7], Equals, "4") - c.Assert(res.Rows()[2][7], Equals, "1") - c.Assert(res.Rows()[3][7], Equals, "4") - c.Assert(res.Rows()[4][7], Equals, "0") + require.Len(t, res.Rows(), 5) + require.Equal(t, "1", res.Rows()[0][7]) + require.Equal(t, "4", res.Rows()[1][7]) + require.Equal(t, "1", res.Rows()[2][7]) + require.Equal(t, "4", res.Rows()[3][7]) + require.Equal(t, "0", res.Rows()[4][7]) } -func (s *testShowStatsSuite) TestShowPartitionStats(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowPartitionStats(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) testkit.WithPruneMode(tk, variable.Static, func() { tk.MustExec("set @@session.tidb_enable_table_partition=1") // Version2 is tested in TestGlobalStatsData1/2/3 and TestAnalyzeGlobalStatsWithOpts. @@ -194,19 +210,19 @@ func (s *testShowStatsSuite) TestShowPartitionStats(c *C) { tk.MustExec("analyze table t") result := tk.MustQuery("show stats_meta") - c.Assert(len(result.Rows()), Equals, 1) - c.Assert(result.Rows()[0][0], Equals, "test") - c.Assert(result.Rows()[0][1], Equals, "t") - c.Assert(result.Rows()[0][2], Equals, "p0") + require.Len(t, result.Rows(), 1) + require.Equal(t, "test", result.Rows()[0][0]) + require.Equal(t, "t", result.Rows()[0][1]) + require.Equal(t, "p0", result.Rows()[0][2]) result = tk.MustQuery("show stats_histograms").Sort() - c.Assert(len(result.Rows()), Equals, 3) - c.Assert(result.Rows()[0][2], Equals, "p0") - c.Assert(result.Rows()[0][3], Equals, "a") - c.Assert(result.Rows()[1][2], Equals, "p0") - c.Assert(result.Rows()[1][3], Equals, "b") - c.Assert(result.Rows()[2][2], Equals, "p0") - c.Assert(result.Rows()[2][3], Equals, "idx") + require.Len(t, result.Rows(), 3) + require.Equal(t, "p0", result.Rows()[0][2]) + require.Equal(t, "a", result.Rows()[0][3]) + require.Equal(t, "p0", result.Rows()[1][2]) + require.Equal(t, "b", result.Rows()[1][3]) + require.Equal(t, "p0", result.Rows()[2][2]) + require.Equal(t, "idx", result.Rows()[2][3]) result = tk.MustQuery("show stats_buckets").Sort() result.Check(testkit.Rows("test t p0 a 0 0 1 1 1 1 0", "test t p0 b 0 0 1 1 1 1 0", "test t p0 idx 1 0 1 1 1 1 0")) @@ -216,55 +232,12 @@ func (s *testShowStatsSuite) TestShowPartitionStats(c *C) { }) } -func (s *testShowStatsSuite) TestShowAnalyzeStatus(c *C) { - tk := testkit.NewTestKit(c, s.store) - statistics.ClearHistoryJobs() - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a int, b int, primary key(a), index idx(b))") - tk.MustExec(`insert into t values (1, 1), (2, 2)`) - - tk.MustExec("set @@tidb_analyze_version=2") - tk.MustExec("analyze table t") - result := tk.MustQuery("show analyze status").Sort() - c.Assert(len(result.Rows()), Equals, 1) - c.Assert(result.Rows()[0][0], Equals, "test") - c.Assert(result.Rows()[0][1], Equals, "t") - c.Assert(result.Rows()[0][2], Equals, "") - c.Assert(result.Rows()[0][3], Equals, "analyze table") - c.Assert(result.Rows()[0][4], Equals, "2") - c.Assert(result.Rows()[0][5], NotNil) - c.Assert(result.Rows()[0][6], NotNil) - c.Assert(result.Rows()[0][7], Equals, "finished") - - statistics.ClearHistoryJobs() - - tk.MustExec("set @@tidb_analyze_version=1") - tk.MustExec("analyze table t") - result = tk.MustQuery("show analyze status").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][0], Equals, "test") - c.Assert(result.Rows()[0][1], Equals, "t") - c.Assert(result.Rows()[0][2], Equals, "") - c.Assert(result.Rows()[0][3], Equals, "analyze columns") - c.Assert(result.Rows()[0][4], Equals, "2") - c.Assert(result.Rows()[0][5], NotNil) - c.Assert(result.Rows()[0][6], NotNil) - c.Assert(result.Rows()[0][7], Equals, "finished") - - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[1][0], Equals, "test") - c.Assert(result.Rows()[1][1], Equals, "t") - c.Assert(result.Rows()[1][2], Equals, "") - c.Assert(result.Rows()[1][3], Equals, "analyze index idx") - c.Assert(result.Rows()[1][4], Equals, "2") - c.Assert(result.Rows()[1][5], NotNil) - c.Assert(result.Rows()[1][6], NotNil) - c.Assert(result.Rows()[1][7], Equals, "finished") -} +func TestShowStatusSnapshot(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() -func (s *testShowStatsSuite) TestShowStatusSnapshot(c *C) { - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists test;") tk.MustExec("create database test;") tk.MustExec("use test;") @@ -285,12 +258,16 @@ func (s *testShowStatsSuite) TestShowStatusSnapshot(c *C) { tk.MustQuery("show table status;").Check(testkit.Rows()) tk.MustExec("set @@tidb_snapshot = '" + snapshotTime.Format("2006-01-02 15:04:05.999999") + "'") result := tk.MustQuery("show table status;") - c.Check(result.Rows()[0][0], Matches, "t") + require.Regexp(t, "t", result.Rows()[0][0]) } -func (s *testShowStatsSuite) TestShowStatsExtended(c *C) { - tk := testkit.NewTestKit(c, s.store) - s.domain.StatsHandle().Clear() +func TestShowStatsExtended(t *testing.T) { + t.Parallel() + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + dom.StatsHandle().Clear() tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int, c int)") @@ -304,7 +281,7 @@ func (s *testShowStatsSuite) TestShowStatsExtended(c *C) { "s2 0", )) result := tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") - c.Assert(len(result.Rows()), Equals, 0) + require.Len(t, result.Rows(), 0) tk.MustExec("analyze table t") tk.MustQuery("select name, status from mysql.stats_extended where name like 's%'").Sort().Check(testkit.Rows( @@ -312,20 +289,21 @@ func (s *testShowStatsSuite) TestShowStatsExtended(c *C) { "s2 1", )) result = tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][0], Equals, "test") - c.Assert(result.Rows()[0][1], Equals, "t") - c.Assert(result.Rows()[0][2], Equals, "s1") - c.Assert(result.Rows()[0][3], Equals, "[a,b]") - c.Assert(result.Rows()[0][4], Equals, "correlation") - c.Assert(result.Rows()[0][5], Equals, "1.000000") - c.Assert(result.Rows()[1][0], Equals, "test") - c.Assert(result.Rows()[1][1], Equals, "t") - c.Assert(result.Rows()[1][2], Equals, "s2") - c.Assert(result.Rows()[1][3], Equals, "[a,c]") - c.Assert(result.Rows()[1][4], Equals, "correlation") - c.Assert(result.Rows()[1][5], Equals, "-1.000000") - c.Assert(result.Rows()[1][6], Equals, result.Rows()[0][6]) + require.Len(t, result.Rows(), 2) + require.Equal(t, "test", result.Rows()[0][0]) + require.Equal(t, "t", result.Rows()[0][1]) + require.Equal(t, "s1", result.Rows()[0][2]) + require.Equal(t, "[a,b]", result.Rows()[0][3]) + require.Equal(t, "correlation", result.Rows()[0][4]) + require.Equal(t, "1.000000", result.Rows()[0][5]) + + require.Equal(t, "test", result.Rows()[1][0]) + require.Equal(t, "t", result.Rows()[1][1]) + require.Equal(t, "s2", result.Rows()[1][2]) + require.Equal(t, "[a,c]", result.Rows()[1][3]) + require.Equal(t, "correlation", result.Rows()[1][4]) + require.Equal(t, "-1.000000", result.Rows()[1][5]) + require.Equal(t, result.Rows()[1][6], result.Rows()[0][6]) tk.MustExec("alter table t drop stats_extended s1") tk.MustExec("alter table t drop stats_extended s2") @@ -333,24 +311,28 @@ func (s *testShowStatsSuite) TestShowStatsExtended(c *C) { "s1 2", "s2 2", )) - s.domain.StatsHandle().Update(s.domain.InfoSchema()) + dom.StatsHandle().Update(dom.InfoSchema()) result = tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") - c.Assert(len(result.Rows()), Equals, 0) + require.Len(t, result.Rows(), 0) } -func (s *testShowStatsSuite) TestShowColumnStatsUsage(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowColumnStatsUsage(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 t1, t2") tk.MustExec("create table t1 (a int, b int, index idx_a_b(a, b))") tk.MustExec("create table t2 (a int, b int) partition by range(a) (partition p0 values less than (10), partition p1 values less than (20), partition p2 values less than maxvalue)") - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() t1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - c.Assert(err, IsNil) + require.NoError(t, err) t2, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec(fmt.Sprintf("insert into mysql.column_stats_usage values (%d, %d, null, '2021-10-20 08:00:00')", t1.Meta().ID, t1.Meta().Columns[0].ID)) tk.MustExec(fmt.Sprintf("insert into mysql.column_stats_usage values (%d, %d, '2021-10-20 09:00:00', null)", t2.Meta().ID, t2.Meta().Columns[0].ID)) p0 := t2.Meta().GetPartitionInfo().Definitions[0] @@ -358,12 +340,13 @@ func (s *testShowStatsSuite) TestShowColumnStatsUsage(c *C) { result := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Sort() rows := result.Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0], DeepEquals, []interface{}{"test", "t1", "", t1.Meta().Columns[0].Name.O, "", "2021-10-20 08:00:00"}) + require.Len(t, rows, 1) + require.Equal(t, rows[0], []interface{}{"test", "t1", "", t1.Meta().Columns[0].Name.O, "", "2021-10-20 08:00:00"}) result = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't2'").Sort() rows = result.Rows() - c.Assert(len(rows), Equals, 2) - c.Assert(rows[0], DeepEquals, []interface{}{"test", "t2", "global", t1.Meta().Columns[0].Name.O, "2021-10-20 09:00:00", ""}) - c.Assert(rows[1], DeepEquals, []interface{}{"test", "t2", p0.Name.O, t1.Meta().Columns[0].Name.O, "2021-10-20 09:00:00", ""}) + + require.Len(t, rows, 2) + require.Equal(t, rows[0], []interface{}{"test", "t2", "global", t1.Meta().Columns[0].Name.O, "2021-10-20 09:00:00", ""}) + require.Equal(t, rows[1], []interface{}{"test", "t2", p0.Name.O, t1.Meta().Columns[0].Name.O, "2021-10-20 09:00:00", ""}) } diff --git a/executor/show_test.go b/executor/show_test.go index 3f0c9a213c9ef..aa877e21ddc93 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -612,6 +612,17 @@ func (s *testSuite5) TestShowOpenTables(c *C) { tk.MustQuery("show open tables") tk.MustQuery("show open tables in test") } +func (s *testSuite5) TestShowCreateViewDefiner(c *C) { + tk := testkit.NewTestKit(c, s.store) + se, err := session.CreateSession4Test(s.store) + c.Assert(err, IsNil) + c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%", AuthUsername: "root", AuthHostname: "%"}, nil, nil), IsTrue) + tk.Se = se + tk.MustExec("use test") + tk.MustExec("create or replace view v1 as select 1") + tk.MustQuery("show create view v1").Check(testutil.RowsWithSep("|", "v1|CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`%` SQL SECURITY DEFINER VIEW `v1` (`1`) AS SELECT 1 AS `1`|utf8mb4|utf8mb4_bin")) + tk.MustExec("drop view v1") +} func (s *testSuite5) TestShowCreateTable(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/executor/simple.go b/executor/simple.go index 519c02a37834b..82e941eca143f 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -661,6 +661,12 @@ func (e *SimpleExec) executeRevokeRole(ctx context.Context, s *ast.RevokeRoleStm return errors.Trace(err) } sql := new(strings.Builder) + // when an active role of current user is revoked, + // it should be removed from activeRoles + activeRoles, curUser, curHost := e.ctx.GetSessionVars().ActiveRoles, "", "" + if user := e.ctx.GetSessionVars().User; user != nil { + curUser, curHost = user.AuthUsername, user.AuthHostname + } for _, user := range s.Users { exists, err := userExists(ctx, e.ctx, user.Username, user.Hostname) if err != nil { @@ -693,11 +699,29 @@ func (e *SimpleExec) executeRevokeRole(ctx context.Context, s *ast.RevokeRoleStm } return ErrCannotUser.GenWithStackByArgs("REVOKE ROLE", role.String()) } + + // delete from activeRoles + if curUser == user.Username && curHost == user.Hostname { + for i := 0; i < len(activeRoles); i++ { + if activeRoles[i].Username == role.Username && activeRoles[i].Hostname == role.Hostname { + activeRoles = append(activeRoles[:i], activeRoles[i+1:]...) + break + } + } + } } } if _, err := sqlExecutor.ExecuteInternal(context.TODO(), "commit"); err != nil { return err } + checker := privilege.GetPrivilegeManager(e.ctx) + if checker == nil { + return errors.New("miss privilege checker") + } + if ok, roleName := checker.ActiveRoles(e.ctx, activeRoles); !ok { + u := e.ctx.GetSessionVars().User + return ErrRoleNotGranted.GenWithStackByArgs(roleName, u.String()) + } return domain.GetDomain(e.ctx).NotifyUpdatePrivilege() } diff --git a/executor/simple_test.go b/executor/simple_test.go index 271b51d591008..b8dc034076ec7 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -974,3 +974,17 @@ func (s *testSuite3) TestShowGrantsAfterDropRole(c *C) { tk.MustExec("DROP ROLE r29473") tk.MustQuery("SHOW GRANTS").Check(testkit.Rows("GRANT CREATE USER ON *.* TO 'u29473'@'%'")) } + +func (s *testSuite3) TestDropRoleAfterRevoke(c *C) { + // issue 29781 + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil) + + tk.MustExec("create role r1, r2, r3;") + defer tk.MustExec("drop role if exists r1, r2, r3;") + tk.MustExec("grant r1,r2,r3 to current_user();") + tk.MustExec("set role all;") + tk.MustExec("revoke r1, r3 from root;") + tk.MustExec("drop role r1;") +} diff --git a/executor/slow_query_test.go b/executor/slow_query_test.go index feb8ac60ba4c1..66ca71897602f 100644 --- a/executor/slow_query_test.go +++ b/executor/slow_query_test.go @@ -54,7 +54,7 @@ func parseLog(retriever *slowQueryRetriever, sctx sessionctx.Context, reader *bu } func newSlowQueryRetriever() (*slowQueryRetriever, error) { - newISBuilder, err := infoschema.NewBuilder(nil).InitWithDBInfos(nil, nil, nil, 0) + newISBuilder, err := infoschema.NewBuilder(nil, nil).InitWithDBInfos(nil, nil, nil, 0) if err != nil { return nil, err } diff --git a/executor/testdata/executor_suite_in.json b/executor/testdata/executor_suite_in.json index cd8fa234c0117..c8d609cdf8e31 100644 --- a/executor/testdata/executor_suite_in.json +++ b/executor/testdata/executor_suite_in.json @@ -40,6 +40,25 @@ "SELECT * FROM t1 NATURAL LEFT JOIN t2 WHERE not(t1.a <=> t2.a)" ] }, + { + "name": "TestUsingAndNaturalJoinSchema", + "cases": [ + "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) natural join (t3 natural join t4);", + "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) join (t3 natural join t4) using (b);", + "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) join (t3 natural join t4) using (c);", + "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) join (t3 natural join t4) using (c,b);", + "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) left outer join (t3 natural join t4) using (b);", + "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) left outer join (t3 natural join t4) using (c);", + "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) left outer join (t3 natural join t4) using (c,b);", + "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) right outer join (t3 natural join t4) using (b);", + "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) right outer join (t3 natural join t4) using (c);", + "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) right outer join (t3 natural join t4) using (c,b);", + "select * from (t1 natural join t2) natural join (t3 natural join t4);", + "select * from (t1 natural join t2) join (t3 natural join t4) using (b);", + "select * from (t1 natural join t2) left outer join (t3 natural join t4) using (b);", + "select * from (t1 natural join t2) right outer join (t3 natural join t4) using (c,b);" + ] + }, { "name": "TestIndexScanWithYearCol", "cases": [ diff --git a/executor/testdata/executor_suite_out.json b/executor/testdata/executor_suite_out.json index eab098751f2ba..d2ad1621f6049 100644 --- a/executor/testdata/executor_suite_out.json +++ b/executor/testdata/executor_suite_out.json @@ -519,6 +519,121 @@ } ] }, + { + "Name": "TestUsingAndNaturalJoinSchema", + "Cases": [ + { + "SQL": "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) natural join (t3 natural join t4);", + "Res": [ + "3 1 2 1 1 3 11 3", + "3 1 2 1 1 3 2 3" + ] + }, + { + "SQL": "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) join (t3 natural join t4) using (b);", + "Res": [ + "10 1 2 1 1 3 11 3", + "10 1 2 1 1 3 2 3", + "3 1 2 1 1 3 11 3", + "3 1 2 1 1 3 2 3" + ] + }, + { + "SQL": "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) join (t3 natural join t4) using (c);", + "Res": [ + "3 1 2 1 1 3 11 3", + "3 1 2 1 1 3 2 3" + ] + }, + { + "SQL": "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) join (t3 natural join t4) using (c,b);", + "Res": [ + "3 1 2 1 1 3 11 3", + "3 1 2 1 1 3 2 3" + ] + }, + { + "SQL": "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) left outer join (t3 natural join t4) using (b);", + "Res": [ + "10 1 2 1 1 3 11 3", + "10 1 2 1 1 3 2 3", + "3 1 2 1 1 3 11 3", + "3 1 2 1 1 3 2 3" + ] + }, + { + "SQL": "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) left outer join (t3 natural join t4) using (c);", + "Res": [ + "10 1 2 1 ", + "3 1 2 1 1 3 11 3", + "3 1 2 1 1 3 2 3" + ] + }, + { + "SQL": "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) left outer join (t3 natural join t4) using (c,b);", + "Res": [ + "10 1 2 1 ", + "3 1 2 1 1 3 11 3", + "3 1 2 1 1 3 2 3" + ] + }, + { + "SQL": "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) right outer join (t3 natural join t4) using (b);", + "Res": [ + "10 1 2 1 1 3 11 3", + "10 1 2 1 1 3 2 3", + "3 1 2 1 1 3 11 3", + "3 1 2 1 1 3 2 3" + ] + }, + { + "SQL": "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) right outer join (t3 natural join t4) using (c);", + "Res": [ + "3 1 2 1 1 3 11 3", + "3 1 2 1 1 3 2 3" + ] + }, + { + "SQL": "select t1.*, t2.*, t3.*, t4.* from (t1 natural join t2) right outer join (t3 natural join t4) using (c,b);", + "Res": [ + "3 1 2 1 1 3 11 3", + "3 1 2 1 1 3 2 3" + ] + }, + { + "SQL": "select * from (t1 natural join t2) natural join (t3 natural join t4);", + "Res": [ + "1 3 2 11", + "1 3 2 2" + ] + }, + { + "SQL": "select * from (t1 natural join t2) join (t3 natural join t4) using (b);", + "Res": [ + "1 10 2 3 11", + "1 10 2 3 2", + "1 3 2 3 11", + "1 3 2 3 2" + ] + }, + { + "SQL": "select * from (t1 natural join t2) left outer join (t3 natural join t4) using (b);", + "Res": [ + "1 10 2 3 11", + "1 10 2 3 2", + "1 3 2 3 11", + "1 3 2 3 2" + ] + }, + { + "SQL": "select * from (t1 natural join t2) right outer join (t3 natural join t4) using (c,b);", + "Res": [ + "3 1 11 2", + "3 1 2 2" + ] + } + ] + }, { "Name": "TestIndexScanWithYearCol", "Cases": [ diff --git a/executor/update.go b/executor/update.go index 03a9979878ab1..7df144b28196c 100644 --- a/executor/update.go +++ b/executor/update.go @@ -274,7 +274,7 @@ func (e *UpdateExec) updateRows(ctx context.Context) (int, error) { if variable.TopSQLEnabled() { txn, err := e.ctx.Txn(true) if err == nil { - txn.SetOption(kv.ResourceGroupTag, e.ctx.GetSessionVars().StmtCtx.GetResourceGroupTag()) + txn.SetOption(kv.ResourceGroupTagger, e.ctx.GetSessionVars().StmtCtx.GetResourceGroupTagger()) } } for rowIdx := 0; rowIdx < chk.NumRows(); rowIdx++ { diff --git a/executor/write_serial_test.go b/executor/write_serial_test.go new file mode 100644 index 0000000000000..440ecbeb177e2 --- /dev/null +++ b/executor/write_serial_test.go @@ -0,0 +1,389 @@ +// 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 ( + "testing" + + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/collate" + "github.com/stretchr/testify/require" +) + +func TestUpdate(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + fillData(tk, "update_test") + + updateStr := `UPDATE update_test SET name = "abc" where id > 0;` + tk.MustExec(updateStr) + tk.CheckExecResult(2, 0) + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") + + // select data + tk.MustExec("begin") + r := tk.MustQuery(`SELECT * from update_test limit 2;`) + r.Check(testkit.Rows("1 abc", "2 abc")) + tk.MustExec("commit") + + tk.MustExec(`UPDATE update_test SET name = "foo"`) + tk.CheckExecResult(2, 0) + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") + + // table option is auto-increment + tk.MustExec("begin") + tk.MustExec("drop table if exists update_test;") + tk.MustExec("commit") + tk.MustExec("begin") + tk.MustExec("create table update_test(id int not null auto_increment, name varchar(255), primary key(id))") + tk.MustExec("insert into update_test(name) values ('aa')") + tk.MustExec("update update_test set id = 8 where name = 'aa'") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") + tk.MustExec("insert into update_test(name) values ('bb')") + tk.MustExec("commit") + tk.MustExec("begin") + r = tk.MustQuery("select * from update_test;") + r.Check(testkit.Rows("8 aa", "9 bb")) + tk.MustExec("commit") + + tk.MustExec("begin") + tk.MustExec("drop table if exists update_test;") + tk.MustExec("commit") + tk.MustExec("begin") + tk.MustExec("create table update_test(id int not null auto_increment, name varchar(255), index(id))") + tk.MustExec("insert into update_test(name) values ('aa')") + _, err := tk.Exec("update update_test set id = null where name = 'aa'") + require.EqualError(t, err, "[table:1048]Column 'id' cannot be null") + + tk.MustExec("drop table update_test") + tk.MustExec("create table update_test(id int)") + tk.MustExec("begin") + tk.MustExec("insert into update_test(id) values (1)") + tk.MustExec("update update_test set id = 2 where id = 1 limit 1") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") + r = tk.MustQuery("select * from update_test;") + r.Check(testkit.Rows("2")) + tk.MustExec("commit") + + // Test that in a transaction, when a constraint failed in an update statement, the record is not inserted. + tk.MustExec("create table update_unique (id int primary key, name int unique)") + tk.MustExec("insert update_unique values (1, 1), (2, 2);") + tk.MustExec("begin") + _, err = tk.Exec("update update_unique set name = 1 where id = 2") + require.Error(t, err) + tk.MustExec("commit") + tk.MustQuery("select * from update_unique").Check(testkit.Rows("1 1", "2 2")) + + // test update ignore for pimary key + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a bigint, primary key (a));") + tk.MustExec("insert into t values (1)") + tk.MustExec("insert into t values (2)") + _, err = tk.Exec("update ignore t set a = 1 where a = 2;") + require.NoError(t, err) + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 1") + r = tk.MustQuery("SHOW WARNINGS;") + r.Check(testkit.Rows("Warning 1062 Duplicate entry '1' for key 'PRIMARY'")) + tk.MustQuery("select * from t").Check(testkit.Rows("1", "2")) + + // test update ignore for truncate as warning + _, err = tk.Exec("update ignore t set a = 1 where a = (select '2a')") + require.NoError(t, err) + r = tk.MustQuery("SHOW WARNINGS;") + r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1062 Duplicate entry '1' for key 'PRIMARY'")) + + tk.MustExec("update ignore t set a = 42 where a = 2;") + tk.MustQuery("select * from t").Check(testkit.Rows("1", "42")) + + // test update ignore for unique key + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a bigint, unique key I_uniq (a));") + tk.MustExec("insert into t values (1)") + tk.MustExec("insert into t values (2)") + _, err = tk.Exec("update ignore t set a = 1 where a = 2;") + require.NoError(t, err) + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 1") + r = tk.MustQuery("SHOW WARNINGS;") + r.Check(testkit.Rows("Warning 1062 Duplicate entry '1' for key 'I_uniq'")) + tk.MustQuery("select * from t").Check(testkit.Rows("1", "2")) + + // test issue21965 + tk.MustExec("drop table if exists t;") + tk.MustExec("set @@session.tidb_enable_list_partition = ON") + tk.MustExec("create table t (a int) partition by list (a) (partition p0 values in (0,1));") + tk.MustExec("insert ignore into t values (1);") + tk.MustExec("update ignore t set a=2 where a=1;") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 0") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int key) partition by list (a) (partition p0 values in (0,1));") + tk.MustExec("insert ignore into t values (1);") + tk.MustExec("update ignore t set a=2 where a=1;") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 0") + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id integer auto_increment, t1 datetime, t2 datetime, primary key (id))") + tk.MustExec("insert into t(t1, t2) values('2000-10-01 01:01:01', '2017-01-01 10:10:10')") + tk.MustQuery("select * from t").Check(testkit.Rows("1 2000-10-01 01:01:01 2017-01-01 10:10:10")) + tk.MustExec("update t set t1 = '2017-10-01 10:10:11', t2 = date_add(t1, INTERVAL 10 MINUTE) where id = 1") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") + tk.MustQuery("select * from t").Check(testkit.Rows("1 2017-10-01 10:10:11 2000-10-01 01:11:01")) + + // for issue #5132 + tk.MustExec("CREATE TABLE `tt1` (" + + "`a` int(11) NOT NULL," + + "`b` varchar(32) DEFAULT NULL," + + "`c` varchar(32) DEFAULT NULL," + + "PRIMARY KEY (`a`)," + + "UNIQUE KEY `b_idx` (`b`)" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin;") + tk.MustExec("insert into tt1 values(1, 'a', 'a');") + tk.MustExec("insert into tt1 values(2, 'd', 'b');") + r = tk.MustQuery("select * from tt1;") + r.Check(testkit.Rows("1 a a", "2 d b")) + tk.MustExec("update tt1 set a=5 where c='b';") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") + r = tk.MustQuery("select * from tt1;") + r.Check(testkit.Rows("1 a a", "5 d b")) + + // Automatic Updating for TIMESTAMP + tk.MustExec("CREATE TABLE `tsup` (" + + "`a` int," + + "`ts` TIMESTAMP DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP," + + "KEY `idx` (`ts`)" + + ");") + tk.MustExec("set @orig_sql_mode=@@sql_mode; set @@sql_mode='';") + tk.MustExec("insert into tsup values(1, '0000-00-00 00:00:00');") + tk.MustExec("update tsup set a=5;") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") + r1 := tk.MustQuery("select ts from tsup use index (idx);") + r2 := tk.MustQuery("select ts from tsup;") + r1.Check(r2.Rows()) + tk.MustExec("update tsup set ts='2019-01-01';") + tk.MustQuery("select ts from tsup;").Check(testkit.Rows("2019-01-01 00:00:00")) + tk.MustExec("set @@sql_mode=@orig_sql_mode;") + + // issue 5532 + tk.MustExec("create table decimals (a decimal(20, 0) not null)") + tk.MustExec("insert into decimals values (201)") + // A warning rather than data truncated error. + tk.MustExec("update decimals set a = a + 1.23;") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 1") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect DECIMAL value: '202.23'")) + r = tk.MustQuery("select * from decimals") + r.Check(testkit.Rows("202")) + + tk.MustExec("drop table t") + tk.MustExec("CREATE TABLE `t` ( `c1` year DEFAULT NULL, `c2` year DEFAULT NULL, `c3` date DEFAULT NULL, `c4` datetime DEFAULT NULL, KEY `idx` (`c1`,`c2`))") + _, err = tk.Exec("UPDATE t SET c2=16777215 WHERE c1>= -8388608 AND c1 < -9 ORDER BY c1 LIMIT 2") + require.NoError(t, err) + + tk.MustGetErrCode("update (select * from t) t set c1 = 1111111", mysql.ErrNonUpdatableTable) + + // test update ignore for bad null error + tk.MustExec("drop table if exists t;") + tk.MustExec(`create table t (i int not null default 10)`) + tk.MustExec("insert into t values (1)") + tk.MustExec("update ignore t set i = null;") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 1") + r = tk.MustQuery("SHOW WARNINGS;") + r.Check(testkit.Rows("Warning 1048 Column 'i' cannot be null")) + tk.MustQuery("select * from t").Check(testkit.Rows("0")) + + // issue 7237, update subquery table should be forbidden + tk.MustExec("drop table t") + tk.MustExec("create table t (k int, v int)") + _, err = tk.Exec("update t, (select * from t) as b set b.k = t.k") + require.EqualError(t, err, "[planner:1288]The target table b of the UPDATE is not updatable") + tk.MustExec("update t, (select * from t) as b set t.k = b.k") + + // issue 8045 + tk.MustExec("drop table if exists t1") + tk.MustExec(`CREATE TABLE t1 (c1 float)`) + tk.MustExec("INSERT INTO t1 SET c1 = 1") + tk.MustExec("UPDATE t1 SET c1 = 1.2 WHERE c1=1;") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") + + // issue 8119 + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (c1 float(1,1));") + tk.MustExec("insert into t values (0.0);") + _, err = tk.Exec("update t set c1 = 2.0;") + require.True(t, types.ErrWarnDataOutOfRange.Equal(err)) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a datetime not null, b datetime)") + tk.MustExec("insert into t value('1999-12-12', '1999-12-13')") + tk.MustExec("set @orig_sql_mode=@@sql_mode; set @@sql_mode='';") + tk.MustQuery("select * from t").Check(testkit.Rows("1999-12-12 00:00:00 1999-12-13 00:00:00")) + tk.MustExec("update t set a = ''") + tk.MustQuery("select * from t").Check(testkit.Rows("0000-00-00 00:00:00 1999-12-13 00:00:00")) + tk.MustExec("update t set b = ''") + tk.MustQuery("select * from t").Check(testkit.Rows("0000-00-00 00:00:00 0000-00-00 00:00:00")) + tk.MustExec("set @@sql_mode=@orig_sql_mode;") + + tk.MustExec("create view v as select * from t") + _, err = tk.Exec("update v set a = '2000-11-11'") + require.EqualError(t, err, core.ErrViewInvalid.GenWithStackByArgs("test", "v").Error()) + tk.MustExec("drop view v") + + tk.MustExec("create sequence seq") + tk.MustGetErrCode("update seq set minvalue=1", mysql.ErrBadField) + tk.MustExec("drop sequence seq") + + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int, b int, c int, d int, e int, index idx(a))") + tk.MustExec("create table t2(a int, b int, c int)") + tk.MustExec("update t1 join t2 on t1.a=t2.a set t1.a=1 where t2.b=1 and t2.c=2") + + // Assign `DEFAULT` in `UPDATE` statement + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("create table t1 (a int default 1, b int default 2);") + tk.MustExec("insert into t1 values (10, 10), (20, 20);") + tk.MustExec("update t1 set a=default where b=10;") + tk.MustQuery("select * from t1;").Check(testkit.Rows("1 10", "20 20")) + tk.MustExec("update t1 set a=30, b=default where a=20;") + tk.MustQuery("select * from t1;").Check(testkit.Rows("1 10", "30 2")) + tk.MustExec("update t1 set a=default, b=default where a=30;") + tk.MustQuery("select * from t1;").Check(testkit.Rows("1 10", "1 2")) + tk.MustExec("insert into t1 values (40, 40)") + tk.MustExec("update t1 set a=default, b=default") + tk.MustQuery("select * from t1;").Check(testkit.Rows("1 2", "1 2", "1 2")) + tk.MustExec("update t1 set a=default(b), b=default(a)") + tk.MustQuery("select * from t1;").Check(testkit.Rows("2 1", "2 1", "2 1")) + // With generated columns + tk.MustExec("create table t2 (a int default 1, b int generated always as (-a) virtual, c int generated always as (-a) stored);") + tk.MustExec("insert into t2 values (10, default, default), (20, default, default)") + tk.MustExec("update t2 set b=default;") + tk.MustQuery("select * from t2;").Check(testkit.Rows("10 -10 -10", "20 -20 -20")) + tk.MustExec("update t2 set a=30, b=default where a=10;") + tk.MustQuery("select * from t2;").Check(testkit.Rows("30 -30 -30", "20 -20 -20")) + tk.MustExec("update t2 set c=default, a=40 where c=-20;") + tk.MustQuery("select * from t2;").Check(testkit.Rows("30 -30 -30", "40 -40 -40")) + tk.MustExec("update t2 set a=default, b=default, c=default where b=-30;") + tk.MustQuery("select * from t2;").Check(testkit.Rows("1 -1 -1", "40 -40 -40")) + tk.MustExec("update t2 set a=default(a), b=default, c=default;") + tk.MustQuery("select * from t2;").Check(testkit.Rows("1 -1 -1", "1 -1 -1")) + tk.MustGetErrCode("update t2 set b=default(a);", mysql.ErrBadGeneratedColumn) + tk.MustGetErrCode("update t2 set a=default(b), b=default(b);", mysql.ErrBadGeneratedColumn) + tk.MustGetErrCode("update t2 set a=default(a), c=default(c);", mysql.ErrBadGeneratedColumn) + tk.MustGetErrCode("update t2 set a=default(a), c=default(a);", mysql.ErrBadGeneratedColumn) + tk.MustExec("drop table t1, t2") +} + +func TestListColumnsPartitionWithGlobalIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@session.tidb_enable_list_partition = ON") + // Test generated column with global index + restoreConfig := config.RestoreFunc() + defer restoreConfig() + config.UpdateGlobal(func(conf *config.Config) { + conf.EnableGlobalIndex = true + }) + tableDefs := []string{ + // Test for virtual generated column with global index + `create table t (a varchar(10), b varchar(1) GENERATED ALWAYS AS (substr(a,1,1)) VIRTUAL) partition by list columns(b) (partition p0 values in ('a','c'), partition p1 values in ('b','d'));`, + // Test for stored generated column with global index + `create table t (a varchar(10), b varchar(1) GENERATED ALWAYS AS (substr(a,1,1)) STORED) partition by list columns(b) (partition p0 values in ('a','c'), partition p1 values in ('b','d'));`, + } + for _, tbl := range tableDefs { + tk.MustExec("drop table if exists t") + tk.MustExec(tbl) + tk.MustExec("alter table t add unique index (a)") + tk.MustExec("insert into t (a) values ('aaa'),('abc'),('acd')") + tk.MustQuery("select a from t partition (p0) order by a").Check(testkit.Rows("aaa", "abc", "acd")) + tk.MustQuery("select * from t where a = 'abc' order by a").Check(testkit.Rows("abc a")) + tk.MustExec("update t set a='bbb' where a = 'aaa'") + tk.MustExec("admin check table t") + tk.MustQuery("select a from t order by a").Check(testkit.Rows("abc", "acd", "bbb")) + // TODO: fix below test. + //tk.MustQuery("select a from t partition (p0) order by a").Check(testkit.Rows("abc", "acd")) + //tk.MustQuery("select a from t partition (p1) order by a").Check(testkit.Rows("bbb")) + tk.MustQuery("select * from t where a = 'bbb' order by a").Check(testkit.Rows("bbb b")) + // Test insert meet duplicate error. + _, err := tk.Exec("insert into t (a) values ('abc')") + require.Error(t, err) + // Test insert on duplicate update + tk.MustExec("insert into t (a) values ('abc') on duplicate key update a='bbc'") + tk.MustQuery("select a from t order by a").Check(testkit.Rows("acd", "bbb", "bbc")) + tk.MustQuery("select * from t where a = 'bbc'").Check(testkit.Rows("bbc b")) + // TODO: fix below test. + //tk.MustQuery("select a from t partition (p0) order by a").Check(testkit.Rows("acd")) + //tk.MustQuery("select a from t partition (p1) order by a").Check(testkit.Rows("bbb", "bbc")) + } +} + +func TestIssue20724(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1(a varchar(10) collate utf8mb4_general_ci)") + tk.MustExec("insert into t1 values ('a')") + tk.MustExec("update t1 set a = 'A'") + tk.MustQuery("select * from t1").Check(testkit.Rows("A")) + tk.MustExec("drop table t1") +} + +func TestIssue20840(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.MustExec("create table t1 (i varchar(20) unique key) collate=utf8mb4_general_ci") + tk.MustExec("insert into t1 values ('a')") + tk.MustExec("replace into t1 values ('A')") + tk.MustQuery("select * from t1").Check(testkit.Rows("A")) + tk.MustExec("drop table t1") +} + +func TestIssueInsertPrefixIndexForNonUTF8Collation(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2, t3") + tk.MustExec("create table t1 ( c_int int, c_str varchar(40) character set ascii collate ascii_bin, primary key(c_int, c_str(8)) clustered , unique key(c_str))") + tk.MustExec("create table t2 ( c_int int, c_str varchar(40) character set latin1 collate latin1_bin, primary key(c_int, c_str(8)) clustered , unique key(c_str))") + tk.MustExec("insert into t1 values (3, 'fervent brattain')") + tk.MustExec("insert into t2 values (3, 'fervent brattain')") + tk.MustExec("admin check table t1") + tk.MustExec("admin check table t2") + + tk.MustExec("create table t3 (x varchar(40) CHARACTER SET ascii COLLATE ascii_bin, UNIQUE KEY uk(x(4)))") + tk.MustExec("insert into t3 select 'abc '") + tk.MustGetErrCode("insert into t3 select 'abc d'", 1062) +} diff --git a/executor/write_test.go b/executor/write_test.go index 5fe78511237da..879c8c76c035f 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -19,10 +19,8 @@ import ( "errors" "fmt" "strconv" - "sync" + "testing" - . "github.com/pingcap/check" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" @@ -31,93 +29,91 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" ) -type testBypassSuite struct{} - -func (s *testBypassSuite) SetUpSuite(c *C) { -} - -func (s *testSuite) TestInsert(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsert(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") testSQL := `drop table if exists insert_test;create table insert_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1);` tk.MustExec(testSQL) testSQL = `insert insert_test (c1) values (1),(2),(NULL);` tk.MustExec(testSQL) - tk.CheckLastMessage("Records: 3 Duplicates: 0 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Records: 3 Duplicates: 0 Warnings: 0") errInsertSelectSQL := `insert insert_test (c1) values ();` tk.MustExec("begin") _, err := tk.Exec(errInsertSelectSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errInsertSelectSQL = `insert insert_test (c1, c2) values (1,2),(1);` tk.MustExec("begin") _, err = tk.Exec(errInsertSelectSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errInsertSelectSQL = `insert insert_test (xxx) values (3);` tk.MustExec("begin") _, err = tk.Exec(errInsertSelectSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errInsertSelectSQL = `insert insert_test_xxx (c1) values ();` tk.MustExec("begin") _, err = tk.Exec(errInsertSelectSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") insertSetSQL := `insert insert_test set c1 = 3;` tk.MustExec(insertSetSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) errInsertSelectSQL = `insert insert_test set c1 = 4, c1 = 5;` tk.MustExec("begin") _, err = tk.Exec(errInsertSelectSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errInsertSelectSQL = `insert insert_test set xxx = 6;` tk.MustExec("begin") _, err = tk.Exec(errInsertSelectSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") insertSelectSQL := `create table insert_test_1 (id int, c1 int);` tk.MustExec(insertSelectSQL) insertSelectSQL = `insert insert_test_1 select id, c1 from insert_test;` tk.MustExec(insertSelectSQL) - tk.CheckLastMessage("Records: 4 Duplicates: 0 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Records: 4 Duplicates: 0 Warnings: 0") insertSelectSQL = `create table insert_test_2 (id int, c1 int);` tk.MustExec(insertSelectSQL) insertSelectSQL = `insert insert_test_1 select id, c1 from insert_test union select id * 10, c1 * 10 from insert_test;` tk.MustExec(insertSelectSQL) - tk.CheckLastMessage("Records: 8 Duplicates: 0 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Records: 8 Duplicates: 0 Warnings: 0") errInsertSelectSQL = `insert insert_test_1 select c1 from insert_test;` tk.MustExec("begin") _, err = tk.Exec(errInsertSelectSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errInsertSelectSQL = `insert insert_test_1 values(default, default, default, default, default)` tk.MustExec("begin") _, err = tk.Exec(errInsertSelectSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") // Updating column is PK handle. @@ -127,22 +123,22 @@ func (s *testSuite) TestInsert(c *C) { r.Check(testkit.Rows(rowStr)) insertSQL := `insert into insert_test (id, c3) values (1, 2) on duplicate key update id=values(id), c2=10;` tk.MustExec(insertSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) r = tk.MustQuery("select * from insert_test where id = 1;") rowStr = fmt.Sprintf("%v %v %v %v", "1", "1", "10", "1") r.Check(testkit.Rows(rowStr)) insertSQL = `insert into insert_test (id, c2) values (1, 1) on duplicate key update insert_test.c2=10;` tk.MustExec(insertSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) _, err = tk.Exec(`insert into insert_test (id, c2) values(1, 1) on duplicate key update t.c2 = 10`) - c.Assert(err, NotNil) + require.Error(t, err) // for on duplicate key insertSQL = `INSERT INTO insert_test (id, c3) VALUES (1, 2) ON DUPLICATE KEY UPDATE c3=values(c3)+c3+3;` tk.MustExec(insertSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) r = tk.MustQuery("select * from insert_test where id = 1;") rowStr = fmt.Sprintf("%v %v %v %v", "1", "1", "10", "6") r.Check(testkit.Rows(rowStr)) @@ -150,21 +146,21 @@ func (s *testSuite) TestInsert(c *C) { // for on duplicate key with ignore insertSQL = `INSERT IGNORE INTO insert_test (id, c3) VALUES (1, 2) ON DUPLICATE KEY UPDATE c3=values(c3)+c3+3;` tk.MustExec(insertSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) r = tk.MustQuery("select * from insert_test where id = 1;") rowStr = fmt.Sprintf("%v %v %v %v", "1", "1", "10", "11") r.Check(testkit.Rows(rowStr)) tk.MustExec("create table insert_err (id int, c1 varchar(8))") _, err = tk.Exec("insert insert_err values (1, 'abcdabcdabcd')") - c.Assert(types.ErrDataTooLong.Equal(err), IsTrue) + require.True(t, types.ErrDataTooLong.Equal(err)) _, err = tk.Exec("insert insert_err values (1, '你好,世界')") - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("create table TEST1 (ID INT NOT NULL, VALUE INT DEFAULT NULL, PRIMARY KEY (ID))") _, err = tk.Exec("INSERT INTO TEST1(id,value) VALUE(3,3) on DUPLICATE KEY UPDATE VALUE=4") - c.Assert(err, IsNil) - tk.CheckLastMessage("") + require.NoError(t, err) + require.Empty(t, tk.Session().LastMessage()) tk.MustExec("create table t (id int)") tk.MustExec("insert into t values(1)") @@ -176,21 +172,21 @@ func (s *testSuite) TestInsert(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(c decimal(5, 5))") _, err = tk.Exec("insert into t value(0)") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = tk.Exec("insert into t value(1)") - c.Assert(types.ErrWarnDataOutOfRange.Equal(err), IsTrue) + require.True(t, types.ErrWarnDataOutOfRange.Equal(err)) tk.MustExec("drop table if exists t") tk.MustExec("create table t(c binary(255))") _, err = tk.Exec("insert into t value(1)") - c.Assert(err, IsNil) + require.NoError(t, err) r = tk.MustQuery("select length(c) from t;") r.Check(testkit.Rows("255")) tk.MustExec("drop table if exists t") tk.MustExec("create table t(c varbinary(255))") _, err = tk.Exec("insert into t value(1)") - c.Assert(err, IsNil) + require.NoError(t, err) r = tk.MustQuery("select length(c) from t;") r.Check(testkit.Rows("1")) @@ -200,7 +196,7 @@ func (s *testSuite) TestInsert(c *C) { tk.MustExec("set @origin_time_zone = @@time_zone") tk.MustExec("set @@time_zone = '+08:00'") _, err = tk.Exec("insert into t value(Unix_timestamp('2002-10-27 01:00'))") - c.Assert(err, IsNil) + require.NoError(t, err) r = tk.MustQuery("select * from t;") r.Check(testkit.Rows("1035651600")) tk.MustExec("set @@time_zone = @origin_time_zone") @@ -208,10 +204,9 @@ func (s *testSuite) TestInsert(c *C) { // issue 3832 tk.MustExec("create table t1 (b char(0));") _, err = tk.Exec(`insert into t1 values ("");`) - c.Assert(err, IsNil) + require.NoError(t, err) // issue 3895 - tk = testkit.NewTestKit(c, s.store) tk.MustExec("USE test;") tk.MustExec("DROP TABLE IF EXISTS t;") tk.MustExec("CREATE TABLE t(a DECIMAL(4,2));") @@ -226,7 +221,7 @@ func (s *testSuite) TestInsert(c *C) { tk.MustExec("DROP TABLE IF EXISTS t;") tk.MustExec("CREATE TABLE t(a datetime);") _, err = tk.Exec("INSERT INTO t VALUES('2017-00-00')") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("set sql_mode = ''") tk.MustExec("INSERT INTO t VALUES('2017-00-00')") r = tk.MustQuery("SELECT * FROM t;") @@ -251,7 +246,7 @@ func (s *testSuite) TestInsert(c *C) { tk.MustExec("create table t(a bigint unsigned);") tk.MustExec(" set @orig_sql_mode = @@sql_mode; set @@sql_mode = 'strict_all_tables';") _, err = tk.Exec("insert into t value (-1);") - c.Assert(types.ErrWarnDataOutOfRange.Equal(err), IsTrue) + require.True(t, types.ErrWarnDataOutOfRange.Equal(err)) tk.MustExec("set @@sql_mode = '';") tk.MustExec("insert into t value (-1);") // TODO: the following warning messages are not consistent with MySQL, fix them in the future PRs @@ -278,7 +273,7 @@ func (s *testSuite) TestInsert(c *C) { tk.MustExec("insert into t value(20070219173709.055870), (20070219173709.055), (20070219173709.055870123)") tk.MustQuery("select * from t").Check(testkit.Rows("17:37:09.055870", "17:37:09.055000", "17:37:09.055870")) _, err = tk.Exec("insert into t value(-20070219173709.055870)") - c.Assert(err.Error(), Equals, "[table:1292]Incorrect time value: '-20070219173709.055870' for column 'a' at row 1") + require.EqualError(t, err, "[table:1292]Incorrect time value: '-20070219173709.055870' for column 'a' at row 1") tk.MustExec("drop table if exists t") tk.MustExec("set @@sql_mode=''") @@ -306,16 +301,16 @@ func (s *testSuite) TestInsert(c *C) { tk.MustExec("create view v as select * from t") _, err = tk.Exec("insert into v values(1,2)") - c.Assert(err.Error(), Equals, "insert into view v is not supported now.") + require.EqualError(t, err, "insert into view v is not supported now.") _, err = tk.Exec("replace into v values(1,2)") - c.Assert(err.Error(), Equals, "replace into view v is not supported now.") + require.EqualError(t, err, "replace into view v is not supported now.") tk.MustExec("drop view v") tk.MustExec("create sequence seq") _, err = tk.Exec("insert into seq values()") - c.Assert(err.Error(), Equals, "insert into sequence seq is not supported now.") + require.EqualError(t, err, "insert into sequence seq is not supported now.") _, err = tk.Exec("replace into seq values()") - c.Assert(err.Error(), Equals, "replace into sequence seq is not supported now.") + require.EqualError(t, err, "replace into sequence seq is not supported now.") tk.MustExec("drop sequence seq") // issue 22851 @@ -323,14 +318,17 @@ func (s *testSuite) TestInsert(c *C) { tk.MustExec("create table t(name varchar(255), b int, c int, primary key(name(2)))") tk.MustExec("insert into t(name, b) values(\"cha\", 3)") _, err = tk.Exec("insert into t(name, b) values(\"chb\", 3)") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry 'ch' for key 'PRIMARY'") + require.EqualError(t, err, "[kv:1062]Duplicate entry 'ch' for key 'PRIMARY'") tk.MustExec("insert into t(name, b) values(\"测试\", 3)") _, err = tk.Exec("insert into t(name, b) values(\"测试\", 3)") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '测试' for key 'PRIMARY'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '测试' for key 'PRIMARY'") } -func (s *testSuiteP2) TestMultiBatch(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMultiBatch(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 t,t0") tk.MustExec("create table t0 (i int)") @@ -341,8 +339,11 @@ func (s *testSuiteP2) TestMultiBatch(c *C) { tk.MustExec("admin check table t") } -func (s *testSuite4) TestInsertAutoInc(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertAutoInc(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") createSQL := `drop table if exists insert_autoinc_test; create table insert_autoinc_test (id int primary key auto_increment, c1 int);` tk.MustExec(createSQL) @@ -450,7 +451,7 @@ func (s *testSuite4) TestInsertAutoInc(c *C) { insertSQL = `insert into insert_autoinc_test(id, c1) values (0, 5)` _, err := tk.Exec(insertSQL) // ERROR 1062 (23000): Duplicate entry '0' for key 'PRIMARY' - c.Assert(err, NotNil) + require.Error(t, err) insertSQL = `insert into insert_autoinc_test(c1) values (6)` tk.MustExec(insertSQL) r = tk.MustQuery("select * from insert_autoinc_test;") @@ -474,36 +475,39 @@ func (s *testSuite4) TestInsertAutoInc(c *C) { r.Check(testkit.Rows(rowStr4, rowStr1, rowStr2, rowStr3, rowStr5, rowStr6, rowStr7, rowStr8)) } -func (s *testSuite4) TestInsertIgnore(c *C) { +func TestInsertIgnore(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() var cfg kv.InjectionConfig - tk := testkit.NewTestKit(c, kv.NewInjectedStore(s.store, &cfg)) + tk := testkit.NewTestKit(t, kv.NewInjectedStore(store, &cfg)) tk.MustExec("use test") testSQL := `drop table if exists t; create table t (id int PRIMARY KEY AUTO_INCREMENT, c1 int unique key);` tk.MustExec(testSQL) testSQL = `insert into t values (1, 2);` tk.MustExec(testSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) r := tk.MustQuery("select * from t;") rowStr := fmt.Sprintf("%v %v", "1", "2") r.Check(testkit.Rows(rowStr)) tk.MustExec("insert ignore into t values (1, 3), (2, 3)") - tk.CheckLastMessage("Records: 2 Duplicates: 1 Warnings: 1") + require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 1 Warnings: 1") r = tk.MustQuery("select * from t;") rowStr1 := fmt.Sprintf("%v %v", "2", "3") r.Check(testkit.Rows(rowStr, rowStr1)) tk.MustExec("insert ignore into t values (3, 4), (3, 4)") - tk.CheckLastMessage("Records: 2 Duplicates: 1 Warnings: 1") + require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 1 Warnings: 1") r = tk.MustQuery("select * from t;") rowStr2 := fmt.Sprintf("%v %v", "3", "4") r.Check(testkit.Rows(rowStr, rowStr1, rowStr2)) tk.MustExec("begin") tk.MustExec("insert ignore into t values (4, 4), (4, 5), (4, 6)") - tk.CheckLastMessage("Records: 3 Duplicates: 2 Warnings: 2") + require.Equal(t, tk.Session().LastMessage(), "Records: 3 Duplicates: 2 Warnings: 2") r = tk.MustQuery("select * from t;") rowStr3 := fmt.Sprintf("%v %v", "4", "5") r.Check(testkit.Rows(rowStr, rowStr1, rowStr2, rowStr3)) @@ -511,7 +515,7 @@ func (s *testSuite4) TestInsertIgnore(c *C) { cfg.SetGetError(errors.New("foo")) _, err := tk.Exec("insert ignore into t values (1, 3)") - c.Assert(err, NotNil) + require.Error(t, err) cfg.SetGetError(nil) // for issue 4268 @@ -520,14 +524,14 @@ func (s *testSuite4) TestInsertIgnore(c *C) { tk.MustExec(testSQL) testSQL = "insert ignore into t select '1a';" _, err = tk.Exec(testSQL) - c.Assert(err, IsNil) - tk.CheckLastMessage("Records: 1 Duplicates: 0 Warnings: 1") + require.NoError(t, err) + require.Equal(t, tk.Session().LastMessage(), "Records: 1 Duplicates: 0 Warnings: 1") r = tk.MustQuery("SHOW WARNINGS") r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '1a'")) testSQL = "insert ignore into t values ('1a')" _, err = tk.Exec(testSQL) - c.Assert(err, IsNil) - tk.CheckLastMessage("") + require.NoError(t, err) + require.Empty(t, tk.Session().LastMessage()) r = tk.MustQuery("SHOW WARNINGS") r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '1a'")) @@ -537,10 +541,10 @@ func (s *testSuite4) TestInsertIgnore(c *C) { tk.MustExec(testSQL) testSQL = "insert ignore into t values (1,1);" tk.MustExec(testSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) _, err = tk.Exec(testSQL) - tk.CheckLastMessage("") - c.Assert(err, IsNil) + require.Empty(t, tk.Session().LastMessage()) + require.NoError(t, err) r = tk.MustQuery("SHOW WARNINGS") r.Check(testkit.Rows("Warning 1062 Duplicate entry '1' for key 'PRIMARY'")) @@ -582,7 +586,7 @@ commit;` tk.MustExec(testSQL) testSQL = `insert ignore into badnull values (null)` tk.MustExec(testSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1048 Column 'i' cannot be null")) testSQL = `select * from badnull` tk.MustQuery(testSQL).Check(testkit.Rows("0")) @@ -592,16 +596,19 @@ commit;` tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1526 Table has no partition for value 3")) } -func (s *testSuite8) TestInsertOnDup(c *C) { +func TestInsertOnDup(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() var cfg kv.InjectionConfig - tk := testkit.NewTestKit(c, kv.NewInjectedStore(s.store, &cfg)) + tk := testkit.NewTestKit(t, kv.NewInjectedStore(store, &cfg)) tk.MustExec("use test") testSQL := `drop table if exists t; create table t (i int unique key);` tk.MustExec(testSQL) testSQL = `insert into t values (1),(2);` tk.MustExec(testSQL) - tk.CheckLastMessage("Records: 2 Duplicates: 0 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 0 Warnings: 0") r := tk.MustQuery("select * from t;") rowStr1 := fmt.Sprintf("%v", "1") @@ -609,12 +616,12 @@ func (s *testSuite8) TestInsertOnDup(c *C) { r.Check(testkit.Rows(rowStr1, rowStr2)) tk.MustExec("insert into t values (1), (2) on duplicate key update i = values(i)") - tk.CheckLastMessage("Records: 2 Duplicates: 0 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 0 Warnings: 0") r = tk.MustQuery("select * from t;") r.Check(testkit.Rows(rowStr1, rowStr2)) tk.MustExec("insert into t values (2), (3) on duplicate key update i = 3") - tk.CheckLastMessage("Records: 2 Duplicates: 1 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 1 Warnings: 0") r = tk.MustQuery("select * from t;") rowStr3 := fmt.Sprintf("%v", "3") r.Check(testkit.Rows(rowStr1, rowStr3)) @@ -624,14 +631,14 @@ func (s *testSuite8) TestInsertOnDup(c *C) { tk.MustExec(testSQL) testSQL = `insert into t values (-1, 1);` tk.MustExec(testSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) r = tk.MustQuery("select * from t;") rowStr1 = fmt.Sprintf("%v %v", "-1", "1") r.Check(testkit.Rows(rowStr1)) tk.MustExec("insert into t values (1, 1) on duplicate key update j = values(j)") - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) r = tk.MustQuery("select * from t;") r.Check(testkit.Rows(rowStr1)) @@ -696,13 +703,13 @@ commit;` f2 VARCHAR(5) NOT NULL UNIQUE); INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1);` tk.MustExec(testSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) testSQL = `SELECT LAST_INSERT_ID();` r = tk.MustQuery(testSQL) r.Check(testkit.Rows("1")) testSQL = `INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1);` tk.MustExec(testSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) testSQL = `SELECT LAST_INSERT_ID();` r = tk.MustQuery(testSQL) r.Check(testkit.Rows("1")) @@ -712,19 +719,19 @@ commit;` f2 VARCHAR(5) NOT NULL UNIQUE); INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1);` tk.MustExec(testSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) testSQL = `SELECT LAST_INSERT_ID();` r = tk.MustQuery(testSQL) r.Check(testkit.Rows("1")) testSQL = `INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = LAST_INSERT_ID(f1);` tk.MustExec(testSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) testSQL = `SELECT LAST_INSERT_ID();` r = tk.MustQuery(testSQL) r.Check(testkit.Rows("1")) testSQL = `INSERT t1 (f2) VALUES ('test') ON DUPLICATE KEY UPDATE f1 = 2;` tk.MustExec(testSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) testSQL = `SELECT LAST_INSERT_ID();` r = tk.MustQuery(testSQL) r.Check(testkit.Rows("1")) @@ -733,21 +740,21 @@ commit;` CREATE TABLE t1 (f1 INT); INSERT t1 VALUES (1) ON DUPLICATE KEY UPDATE f1 = 1;` tk.MustExec(testSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) tk.MustQuery(`SELECT * FROM t1;`).Check(testkit.Rows("1")) testSQL = `DROP TABLE IF EXISTS t1; CREATE TABLE t1 (f1 INT PRIMARY KEY, f2 INT NOT NULL UNIQUE); INSERT t1 VALUES (1, 1);` tk.MustExec(testSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) tk.MustExec(`INSERT t1 VALUES (1, 1), (1, 1) ON DUPLICATE KEY UPDATE f1 = 2, f2 = 2;`) - tk.CheckLastMessage("Records: 2 Duplicates: 1 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 1 Warnings: 0") tk.MustQuery(`SELECT * FROM t1 order by f1;`).Check(testkit.Rows("1 1", "2 2")) _, err := tk.Exec(`INSERT t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null;`) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec(`INSERT IGNORE t1 VALUES (1, 1) ON DUPLICATE KEY UPDATE f2 = null;`) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1048 Column 'f2' cannot be null")) tk.MustQuery(`SELECT * FROM t1 order by f1;`).Check(testkit.Rows("1 0", "2 2")) @@ -757,24 +764,27 @@ commit;` tk.MustQuery(`SELECT * FROM t1 order by f1;`).Check(testkit.Rows("1 0", "2 2")) } -func (s *testSuite4) TestInsertIgnoreOnDup(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertIgnoreOnDup(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") testSQL := `drop table if exists t; create table t (i int not null primary key, j int unique key);` tk.MustExec(testSQL) testSQL = `insert into t values (1, 1), (2, 2);` tk.MustExec(testSQL) - tk.CheckLastMessage("Records: 2 Duplicates: 0 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 0 Warnings: 0") testSQL = `insert ignore into t values(1, 1) on duplicate key update i = 2;` tk.MustExec(testSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) testSQL = `select * from t;` r := tk.MustQuery(testSQL) r.Check(testkit.Rows("1 1", "2 2")) testSQL = `insert ignore into t values(1, 1) on duplicate key update j = 2;` tk.MustExec(testSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) testSQL = `select * from t;` r = tk.MustQuery(testSQL) r.Check(testkit.Rows("1 1", "2 2")) @@ -809,8 +819,11 @@ func (s *testSuite4) TestInsertIgnoreOnDup(c *C) { tk.MustQuery("select * from t6").Check(testkit.Rows("100 10 1000")) } -func (s *testSuite4) TestInsertSetWithDefault(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertSetWithDefault(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // Assign `DEFAULT` in `INSERT ... SET ...` statement tk.MustExec("drop table if exists t1, t2;") @@ -856,8 +869,11 @@ func (s *testSuite4) TestInsertSetWithDefault(c *C) { tk.MustExec("drop table t1, t2") } -func (s *testSuite4) TestInsertOnDupUpdateDefault(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertOnDupUpdateDefault(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // Assign `DEFAULT` in `INSERT ... ON DUPLICATE KEY UPDATE ...` statement tk.MustExec("drop table if exists t1, t2;") @@ -895,78 +911,81 @@ func (s *testSuite4) TestInsertOnDupUpdateDefault(c *C) { tk.MustExec("insert into t values (22, 'gold witch'), (24, 'gray singer'), (21, 'silver sight');") tk.MustExec("begin;") err := tk.ExecToErr("insert into t values (21,'black warlock'), (22, 'dark sloth'), (21, 'cyan song') on duplicate key update c_int = c_int + 1, c_string = concat(c_int, ':', c_string);") - c.Assert(kv.ErrKeyExists.Equal(err), IsTrue) + require.True(t, kv.ErrKeyExists.Equal(err)) tk.MustExec("commit;") tk.MustQuery("select * from t order by c_int;").Check(testutil.RowsWithSep("|", "21|silver sight", "22|gold witch", "24|gray singer")) tk.MustExec("drop table t;") } -func (s *testSuite4) TestReplace(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestReplace(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") testSQL := `drop table if exists replace_test; create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1);` tk.MustExec(testSQL) testSQL = `replace replace_test (c1) values (1),(2),(NULL);` tk.MustExec(testSQL) - tk.CheckLastMessage("Records: 3 Duplicates: 0 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Records: 3 Duplicates: 0 Warnings: 0") errReplaceSQL := `replace replace_test (c1) values ();` tk.MustExec("begin") _, err := tk.Exec(errReplaceSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errReplaceSQL = `replace replace_test (c1, c2) values (1,2),(1);` tk.MustExec("begin") _, err = tk.Exec(errReplaceSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errReplaceSQL = `replace replace_test (xxx) values (3);` tk.MustExec("begin") _, err = tk.Exec(errReplaceSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errReplaceSQL = `replace replace_test_xxx (c1) values ();` tk.MustExec("begin") _, err = tk.Exec(errReplaceSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") replaceSetSQL := `replace replace_test set c1 = 3;` tk.MustExec(replaceSetSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) errReplaceSetSQL := `replace replace_test set c1 = 4, c1 = 5;` tk.MustExec("begin") _, err = tk.Exec(errReplaceSetSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errReplaceSetSQL = `replace replace_test set xxx = 6;` tk.MustExec("begin") _, err = tk.Exec(errReplaceSetSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") replaceSelectSQL := `create table replace_test_1 (id int, c1 int);` tk.MustExec(replaceSelectSQL) replaceSelectSQL = `replace replace_test_1 select id, c1 from replace_test;` tk.MustExec(replaceSelectSQL) - tk.CheckLastMessage("Records: 4 Duplicates: 0 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Records: 4 Duplicates: 0 Warnings: 0") replaceSelectSQL = `create table replace_test_2 (id int, c1 int);` tk.MustExec(replaceSelectSQL) replaceSelectSQL = `replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test;` tk.MustExec(replaceSelectSQL) - tk.CheckLastMessage("Records: 8 Duplicates: 0 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Records: 8 Duplicates: 0 Warnings: 0") errReplaceSelectSQL := `replace replace_test_1 select c1 from replace_test;` tk.MustExec("begin") _, err = tk.Exec(errReplaceSelectSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") replaceUniqueIndexSQL := `create table replace_test_3 (c1 int, c2 int, UNIQUE INDEX (c2));` @@ -975,20 +994,20 @@ func (s *testSuite4) TestReplace(c *C) { tk.MustExec(replaceUniqueIndexSQL) replaceUniqueIndexSQL = `replace into replace_test_3 set c2=1;` tk.MustExec(replaceUniqueIndexSQL) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) - tk.CheckLastMessage("") + require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) + require.Empty(t, tk.Session().LastMessage()) replaceUniqueIndexSQL = `replace into replace_test_3 set c1=1, c2=1;` tk.MustExec(replaceUniqueIndexSQL) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(2)) - tk.CheckLastMessage("") + require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) + require.Empty(t, tk.Session().LastMessage()) replaceUniqueIndexSQL = `replace into replace_test_3 set c2=NULL;` tk.MustExec(replaceUniqueIndexSQL) replaceUniqueIndexSQL = `replace into replace_test_3 set c2=NULL;` tk.MustExec(replaceUniqueIndexSQL) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) - tk.CheckLastMessage("") + require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) + require.Empty(t, tk.Session().LastMessage()) replaceUniqueIndexSQL = `create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2));` tk.MustExec(replaceUniqueIndexSQL) @@ -996,8 +1015,8 @@ func (s *testSuite4) TestReplace(c *C) { tk.MustExec(replaceUniqueIndexSQL) replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` tk.MustExec(replaceUniqueIndexSQL) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) - tk.CheckLastMessage("") + require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) + require.Empty(t, tk.Session().LastMessage()) replacePrimaryKeySQL := `create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2));` tk.MustExec(replacePrimaryKeySQL) @@ -1005,18 +1024,18 @@ func (s *testSuite4) TestReplace(c *C) { tk.MustExec(replacePrimaryKeySQL) replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` tk.MustExec(replacePrimaryKeySQL) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) - tk.CheckLastMessage("") + require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) + require.Empty(t, tk.Session().LastMessage()) // For Issue989 issue989SQL := `CREATE TABLE tIssue989 (a int, b int, PRIMARY KEY(a), UNIQUE KEY(b));` tk.MustExec(issue989SQL) issue989SQL = `insert into tIssue989 (a, b) values (1, 2);` tk.MustExec(issue989SQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) issue989SQL = `replace into tIssue989(a, b) values (111, 2);` tk.MustExec(issue989SQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) r := tk.MustQuery("select * from tIssue989;") r.Check(testkit.Rows("111 2")) @@ -1029,8 +1048,8 @@ func (s *testSuite4) TestReplace(c *C) { tk.MustExec(issue1012SQL) issue1012SQL = `replace into tIssue1012(a, b) values (1, 1);` tk.MustExec(issue1012SQL) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(3)) - tk.CheckLastMessage("") + require.Equal(t, int64(3), int64(tk.Session().AffectedRows())) + require.Empty(t, tk.Session().LastMessage()) r = tk.MustQuery("select * from tIssue1012;") r.Check(testkit.Rows("1 1")) @@ -1039,17 +1058,17 @@ func (s *testSuite4) TestReplace(c *C) { tk.MustExec(`create table t1(a int primary key, b int);`) tk.MustExec(`insert into t1 values(1,1),(2,2),(3,3),(4,4),(5,5);`) tk.MustExec(`replace into t1 values(1,1);`) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) - tk.CheckLastMessage("") + require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) + require.Empty(t, tk.Session().LastMessage()) tk.MustExec(`replace into t1 values(1,1),(2,2);`) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(2)) - tk.CheckLastMessage("Records: 2 Duplicates: 0 Warnings: 0") + require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) + require.Equal(t, tk.Session().LastMessage(), "Records: 2 Duplicates: 0 Warnings: 0") tk.MustExec(`replace into t1 values(4,14),(5,15),(6,16),(7,17),(8,18)`) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(7)) - tk.CheckLastMessage("Records: 5 Duplicates: 2 Warnings: 0") + require.Equal(t, int64(7), int64(tk.Session().AffectedRows())) + require.Equal(t, tk.Session().LastMessage(), "Records: 5 Duplicates: 2 Warnings: 0") tk.MustExec(`replace into t1 select * from (select 1, 2) as tmp;`) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(2)) - tk.CheckLastMessage("Records: 1 Duplicates: 1 Warnings: 0") + require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) + require.Equal(t, tk.Session().LastMessage(), "Records: 1 Duplicates: 1 Warnings: 0") // Assign `DEFAULT` in `REPLACE` statement tk.MustExec("drop table if exists t1, t2;") @@ -1082,8 +1101,11 @@ func (s *testSuite4) TestReplace(c *C) { tk.MustExec("drop table t1, t2") } -func (s *testSuite2) TestGeneratedColumnForInsert(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestGeneratedColumnForInsert(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // test cases for default behavior @@ -1131,8 +1153,11 @@ func (s *testSuite2) TestGeneratedColumnForInsert(c *C) { tk.MustQuery(`select * from t1`).Check(testkit.Rows("1000 9.9")) } -func (s *testSuite4) TestPartitionedTableReplace(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPartitionedTableReplace(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") testSQL := `drop table if exists replace_test; create table replace_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1) @@ -1144,46 +1169,46 @@ func (s *testSuite4) TestPartitionedTableReplace(c *C) { tk.MustExec(testSQL) testSQL = `replace replace_test (c1) values (1),(2),(NULL);` tk.MustExec(testSQL) - tk.CheckLastMessage("Records: 3 Duplicates: 0 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Records: 3 Duplicates: 0 Warnings: 0") errReplaceSQL := `replace replace_test (c1) values ();` tk.MustExec("begin") _, err := tk.Exec(errReplaceSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errReplaceSQL = `replace replace_test (c1, c2) values (1,2),(1);` tk.MustExec("begin") _, err = tk.Exec(errReplaceSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errReplaceSQL = `replace replace_test (xxx) values (3);` tk.MustExec("begin") _, err = tk.Exec(errReplaceSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errReplaceSQL = `replace replace_test_xxx (c1) values ();` tk.MustExec("begin") _, err = tk.Exec(errReplaceSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") replaceSetSQL := `replace replace_test set c1 = 3;` tk.MustExec(replaceSetSQL) - tk.CheckLastMessage("") + require.Empty(t, tk.Session().LastMessage()) errReplaceSetSQL := `replace replace_test set c1 = 4, c1 = 5;` tk.MustExec("begin") _, err = tk.Exec(errReplaceSetSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errReplaceSetSQL = `replace replace_test set xxx = 6;` tk.MustExec("begin") _, err = tk.Exec(errReplaceSetSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") tk.MustExec(`drop table if exists replace_test_1`) @@ -1194,7 +1219,7 @@ func (s *testSuite4) TestPartitionedTableReplace(c *C) { PARTITION p3 VALUES LESS THAN (10), PARTITION p4 VALUES LESS THAN (100))`) tk.MustExec(`replace replace_test_1 select id, c1 from replace_test;`) - tk.CheckLastMessage("Records: 4 Duplicates: 0 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Records: 4 Duplicates: 0 Warnings: 0") tk.MustExec(`drop table if exists replace_test_2`) tk.MustExec(`create table replace_test_2 (id int, c1 int) partition by range (id) ( @@ -1203,12 +1228,12 @@ func (s *testSuite4) TestPartitionedTableReplace(c *C) { PARTITION p2 VALUES LESS THAN (100), PARTITION p3 VALUES LESS THAN (300))`) tk.MustExec(`replace replace_test_1 select id, c1 from replace_test union select id * 10, c1 * 10 from replace_test;`) - tk.CheckLastMessage("Records: 8 Duplicates: 0 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Records: 8 Duplicates: 0 Warnings: 0") errReplaceSelectSQL := `replace replace_test_1 select c1 from replace_test;` tk.MustExec("begin") _, err = tk.Exec(errReplaceSelectSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") tk.MustExec(`drop table if exists replace_test_3`) @@ -1221,19 +1246,19 @@ func (s *testSuite4) TestPartitionedTableReplace(c *C) { tk.MustExec(replaceUniqueIndexSQL) replaceUniqueIndexSQL = `replace into replace_test_3 set c2=8;` tk.MustExec(replaceUniqueIndexSQL) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) - tk.CheckLastMessage("") + require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) + require.Empty(t, tk.Session().LastMessage()) replaceUniqueIndexSQL = `replace into replace_test_3 set c1=8, c2=8;` tk.MustExec(replaceUniqueIndexSQL) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(2)) - tk.CheckLastMessage("") + require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) + require.Empty(t, tk.Session().LastMessage()) replaceUniqueIndexSQL = `replace into replace_test_3 set c2=NULL;` tk.MustExec(replaceUniqueIndexSQL) replaceUniqueIndexSQL = `replace into replace_test_3 set c2=NULL;` tk.MustExec(replaceUniqueIndexSQL) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) - tk.CheckLastMessage("") + require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) + require.Empty(t, tk.Session().LastMessage()) replaceUniqueIndexSQL = `create table replace_test_4 (c1 int, c2 int, c3 int, UNIQUE INDEX (c1, c2)) partition by range (c1) ( PARTITION p0 VALUES LESS THAN (4), @@ -1245,7 +1270,7 @@ func (s *testSuite4) TestPartitionedTableReplace(c *C) { tk.MustExec(replaceUniqueIndexSQL) replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` tk.MustExec(replaceUniqueIndexSQL) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) + require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) replacePrimaryKeySQL := `create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)) partition by range (c2) ( PARTITION p0 VALUES LESS THAN (4), @@ -1256,7 +1281,7 @@ func (s *testSuite4) TestPartitionedTableReplace(c *C) { tk.MustExec(replacePrimaryKeySQL) replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` tk.MustExec(replacePrimaryKeySQL) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) + require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) issue989SQL := `CREATE TABLE tIssue989 (a int, b int, KEY(a), UNIQUE KEY(b)) partition by range (b) ( PARTITION p1 VALUES LESS THAN (100), @@ -1270,8 +1295,11 @@ func (s *testSuite4) TestPartitionedTableReplace(c *C) { r.Check(testkit.Rows("111 2")) } -func (s *testSuite4) TestHashPartitionedTableReplace(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestHashPartitionedTableReplace(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_table_partition = '1';") tk.MustExec("drop table if exists replace_test;") @@ -1285,37 +1313,37 @@ func (s *testSuite4) TestHashPartitionedTableReplace(c *C) { errReplaceSQL := `replace replace_test (c1) values ();` tk.MustExec("begin") _, err := tk.Exec(errReplaceSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errReplaceSQL = `replace replace_test (c1, c2) values (1,2),(1);` tk.MustExec("begin") _, err = tk.Exec(errReplaceSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errReplaceSQL = `replace replace_test (xxx) values (3);` tk.MustExec("begin") _, err = tk.Exec(errReplaceSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errReplaceSQL = `replace replace_test_xxx (c1) values ();` tk.MustExec("begin") _, err = tk.Exec(errReplaceSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errReplaceSetSQL := `replace replace_test set c1 = 4, c1 = 5;` tk.MustExec("begin") _, err = tk.Exec(errReplaceSetSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") errReplaceSetSQL = `replace replace_test set xxx = 6;` tk.MustExec("begin") _, err = tk.Exec(errReplaceSetSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") tk.MustExec(`replace replace_test set c1 = 3;`) @@ -1336,7 +1364,7 @@ func (s *testSuite4) TestHashPartitionedTableReplace(c *C) { errReplaceSelectSQL := `replace replace_test_1 select c1 from replace_test;` tk.MustExec("begin") _, err = tk.Exec(errReplaceSelectSQL) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") tk.MustExec(`drop table if exists replace_test_3`) @@ -1345,13 +1373,13 @@ func (s *testSuite4) TestHashPartitionedTableReplace(c *C) { tk.MustExec(`replace into replace_test_3 set c2=8;`) tk.MustExec(`replace into replace_test_3 set c2=8;`) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) + require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) tk.MustExec(`replace into replace_test_3 set c1=8, c2=8;`) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(2)) + require.Equal(t, int64(2), int64(tk.Session().AffectedRows())) tk.MustExec(`replace into replace_test_3 set c2=NULL;`) tk.MustExec(`replace into replace_test_3 set c2=NULL;`) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) + require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) for i := 0; i < 100; i++ { sql := fmt.Sprintf("replace into replace_test_3 set c2=%d;", i) @@ -1367,7 +1395,7 @@ func (s *testSuite4) TestHashPartitionedTableReplace(c *C) { tk.MustExec(replaceUniqueIndexSQL) replaceUniqueIndexSQL = `replace into replace_test_4 set c2=NULL;` tk.MustExec(replaceUniqueIndexSQL) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) + require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) replacePrimaryKeySQL := `create table replace_test_5 (c1 int, c2 int, c3 int, PRIMARY KEY (c1, c2)) partition by hash (c2) partitions 9;` tk.MustExec(replacePrimaryKeySQL) @@ -1375,7 +1403,7 @@ func (s *testSuite4) TestHashPartitionedTableReplace(c *C) { tk.MustExec(replacePrimaryKeySQL) replacePrimaryKeySQL = `replace into replace_test_5 set c1=1, c2=2;` tk.MustExec(replacePrimaryKeySQL) - c.Assert(int64(tk.Se.AffectedRows()), Equals, int64(1)) + require.Equal(t, int64(1), int64(tk.Session().AffectedRows())) issue989SQL := `CREATE TABLE tIssue989 (a int, b int, KEY(a), UNIQUE KEY(b)) partition by hash (b) partitions 10;` tk.MustExec(issue989SQL) @@ -1387,269 +1415,11 @@ func (s *testSuite4) TestHashPartitionedTableReplace(c *C) { r.Check(testkit.Rows("111 2")) } -func (s *testSuite8) TestUpdate(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - s.fillData(tk, "update_test") - - updateStr := `UPDATE update_test SET name = "abc" where id > 0;` - tk.MustExec(updateStr) - tk.CheckExecResult(2, 0) - tk.CheckLastMessage("Rows matched: 2 Changed: 2 Warnings: 0") - - // select data - tk.MustExec("begin") - r := tk.MustQuery(`SELECT * from update_test limit 2;`) - r.Check(testkit.Rows("1 abc", "2 abc")) - tk.MustExec("commit") - - tk.MustExec(`UPDATE update_test SET name = "foo"`) - tk.CheckExecResult(2, 0) - tk.CheckLastMessage("Rows matched: 2 Changed: 2 Warnings: 0") - - // table option is auto-increment - tk.MustExec("begin") - tk.MustExec("drop table if exists update_test;") - tk.MustExec("commit") - tk.MustExec("begin") - tk.MustExec("create table update_test(id int not null auto_increment, name varchar(255), primary key(id))") - tk.MustExec("insert into update_test(name) values ('aa')") - tk.MustExec("update update_test set id = 8 where name = 'aa'") - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 0") - tk.MustExec("insert into update_test(name) values ('bb')") - tk.MustExec("commit") - tk.MustExec("begin") - r = tk.MustQuery("select * from update_test;") - r.Check(testkit.Rows("8 aa", "9 bb")) - tk.MustExec("commit") - - tk.MustExec("begin") - tk.MustExec("drop table if exists update_test;") - tk.MustExec("commit") - tk.MustExec("begin") - tk.MustExec("create table update_test(id int not null auto_increment, name varchar(255), index(id))") - tk.MustExec("insert into update_test(name) values ('aa')") - _, err := tk.Exec("update update_test set id = null where name = 'aa'") - c.Assert(err, NotNil) - c.Assert(err.Error(), DeepEquals, "[table:1048]Column 'id' cannot be null") - - tk.MustExec("drop table update_test") - tk.MustExec("create table update_test(id int)") - tk.MustExec("begin") - tk.MustExec("insert into update_test(id) values (1)") - tk.MustExec("update update_test set id = 2 where id = 1 limit 1") - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 0") - r = tk.MustQuery("select * from update_test;") - r.Check(testkit.Rows("2")) - tk.MustExec("commit") - - // Test that in a transaction, when a constraint failed in an update statement, the record is not inserted. - tk.MustExec("create table update_unique (id int primary key, name int unique)") - tk.MustExec("insert update_unique values (1, 1), (2, 2);") - tk.MustExec("begin") - _, err = tk.Exec("update update_unique set name = 1 where id = 2") - c.Assert(err, NotNil) - tk.MustExec("commit") - tk.MustQuery("select * from update_unique").Check(testkit.Rows("1 1", "2 2")) - - // test update ignore for pimary key - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a bigint, primary key (a));") - tk.MustExec("insert into t values (1)") - tk.MustExec("insert into t values (2)") - _, err = tk.Exec("update ignore t set a = 1 where a = 2;") - c.Assert(err, IsNil) - tk.CheckLastMessage("Rows matched: 1 Changed: 0 Warnings: 1") - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1062 Duplicate entry '1' for key 'PRIMARY'")) - tk.MustQuery("select * from t").Check(testkit.Rows("1", "2")) - - // test update ignore for truncate as warning - _, err = tk.Exec("update ignore t set a = 1 where a = (select '2a')") - c.Assert(err, IsNil) - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1062 Duplicate entry '1' for key 'PRIMARY'")) - - tk.MustExec("update ignore t set a = 42 where a = 2;") - tk.MustQuery("select * from t").Check(testkit.Rows("1", "42")) - - // test update ignore for unique key - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a bigint, unique key I_uniq (a));") - tk.MustExec("insert into t values (1)") - tk.MustExec("insert into t values (2)") - _, err = tk.Exec("update ignore t set a = 1 where a = 2;") - c.Assert(err, IsNil) - tk.CheckLastMessage("Rows matched: 1 Changed: 0 Warnings: 1") - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1062 Duplicate entry '1' for key 'I_uniq'")) - tk.MustQuery("select * from t").Check(testkit.Rows("1", "2")) - - // test issue21965 - tk.MustExec("drop table if exists t;") - tk.MustExec("set @@session.tidb_enable_list_partition = ON") - tk.MustExec("create table t (a int) partition by list (a) (partition p0 values in (0,1));") - tk.MustExec("insert ignore into t values (1);") - tk.MustExec("update ignore t set a=2 where a=1;") - tk.CheckLastMessage("Rows matched: 1 Changed: 0 Warnings: 0") - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (a int key) partition by list (a) (partition p0 values in (0,1));") - tk.MustExec("insert ignore into t values (1);") - tk.MustExec("update ignore t set a=2 where a=1;") - tk.CheckLastMessage("Rows matched: 1 Changed: 0 Warnings: 0") - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(id integer auto_increment, t1 datetime, t2 datetime, primary key (id))") - tk.MustExec("insert into t(t1, t2) values('2000-10-01 01:01:01', '2017-01-01 10:10:10')") - tk.MustQuery("select * from t").Check(testkit.Rows("1 2000-10-01 01:01:01 2017-01-01 10:10:10")) - tk.MustExec("update t set t1 = '2017-10-01 10:10:11', t2 = date_add(t1, INTERVAL 10 MINUTE) where id = 1") - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 0") - tk.MustQuery("select * from t").Check(testkit.Rows("1 2017-10-01 10:10:11 2000-10-01 01:11:01")) - - // for issue #5132 - tk.MustExec("CREATE TABLE `tt1` (" + - "`a` int(11) NOT NULL," + - "`b` varchar(32) DEFAULT NULL," + - "`c` varchar(32) DEFAULT NULL," + - "PRIMARY KEY (`a`)," + - "UNIQUE KEY `b_idx` (`b`)" + - ") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin;") - tk.MustExec("insert into tt1 values(1, 'a', 'a');") - tk.MustExec("insert into tt1 values(2, 'd', 'b');") - r = tk.MustQuery("select * from tt1;") - r.Check(testkit.Rows("1 a a", "2 d b")) - tk.MustExec("update tt1 set a=5 where c='b';") - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 0") - r = tk.MustQuery("select * from tt1;") - r.Check(testkit.Rows("1 a a", "5 d b")) - - // Automatic Updating for TIMESTAMP - tk.MustExec("CREATE TABLE `tsup` (" + - "`a` int," + - "`ts` TIMESTAMP DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP," + - "KEY `idx` (`ts`)" + - ");") - tk.MustExec("set @orig_sql_mode=@@sql_mode; set @@sql_mode='';") - tk.MustExec("insert into tsup values(1, '0000-00-00 00:00:00');") - tk.MustExec("update tsup set a=5;") - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 0") - r1 := tk.MustQuery("select ts from tsup use index (idx);") - r2 := tk.MustQuery("select ts from tsup;") - r1.Check(r2.Rows()) - tk.MustExec("update tsup set ts='2019-01-01';") - tk.MustQuery("select ts from tsup;").Check(testkit.Rows("2019-01-01 00:00:00")) - tk.MustExec("set @@sql_mode=@orig_sql_mode;") - - // issue 5532 - tk.MustExec("create table decimals (a decimal(20, 0) not null)") - tk.MustExec("insert into decimals values (201)") - // A warning rather than data truncated error. - tk.MustExec("update decimals set a = a + 1.23;") - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 1") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect DECIMAL value: '202.23'")) - r = tk.MustQuery("select * from decimals") - r.Check(testkit.Rows("202")) - - tk.MustExec("drop table t") - tk.MustExec("CREATE TABLE `t` ( `c1` year DEFAULT NULL, `c2` year DEFAULT NULL, `c3` date DEFAULT NULL, `c4` datetime DEFAULT NULL, KEY `idx` (`c1`,`c2`))") - _, err = tk.Exec("UPDATE t SET c2=16777215 WHERE c1>= -8388608 AND c1 < -9 ORDER BY c1 LIMIT 2") - c.Assert(err, IsNil) - - tk.MustGetErrCode("update (select * from t) t set c1 = 1111111", mysql.ErrNonUpdatableTable) - - // test update ignore for bad null error - tk.MustExec("drop table if exists t;") - tk.MustExec(`create table t (i int not null default 10)`) - tk.MustExec("insert into t values (1)") - tk.MustExec("update ignore t set i = null;") - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 1") - r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1048 Column 'i' cannot be null")) - tk.MustQuery("select * from t").Check(testkit.Rows("0")) - - // issue 7237, update subquery table should be forbidden - tk.MustExec("drop table t") - tk.MustExec("create table t (k int, v int)") - _, err = tk.Exec("update t, (select * from t) as b set b.k = t.k") - c.Assert(err.Error(), Equals, "[planner:1288]The target table b of the UPDATE is not updatable") - tk.MustExec("update t, (select * from t) as b set t.k = b.k") - - // issue 8045 - tk.MustExec("drop table if exists t1") - tk.MustExec(`CREATE TABLE t1 (c1 float)`) - tk.MustExec("INSERT INTO t1 SET c1 = 1") - tk.MustExec("UPDATE t1 SET c1 = 1.2 WHERE c1=1;") - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 0") - - // issue 8119 - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (c1 float(1,1));") - tk.MustExec("insert into t values (0.0);") - _, err = tk.Exec("update t set c1 = 2.0;") - c.Assert(types.ErrWarnDataOutOfRange.Equal(err), IsTrue) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a datetime not null, b datetime)") - tk.MustExec("insert into t value('1999-12-12', '1999-12-13')") - tk.MustExec("set @orig_sql_mode=@@sql_mode; set @@sql_mode='';") - tk.MustQuery("select * from t").Check(testkit.Rows("1999-12-12 00:00:00 1999-12-13 00:00:00")) - tk.MustExec("update t set a = ''") - tk.MustQuery("select * from t").Check(testkit.Rows("0000-00-00 00:00:00 1999-12-13 00:00:00")) - tk.MustExec("update t set b = ''") - tk.MustQuery("select * from t").Check(testkit.Rows("0000-00-00 00:00:00 0000-00-00 00:00:00")) - tk.MustExec("set @@sql_mode=@orig_sql_mode;") - - tk.MustExec("create view v as select * from t") - _, err = tk.Exec("update v set a = '2000-11-11'") - c.Assert(err.Error(), Equals, core.ErrViewInvalid.GenWithStackByArgs("test", "v").Error()) - tk.MustExec("drop view v") - - tk.MustExec("create sequence seq") - tk.MustGetErrCode("update seq set minvalue=1", mysql.ErrBadField) - tk.MustExec("drop sequence seq") - - tk.MustExec("drop table if exists t1, t2") - tk.MustExec("create table t1(a int, b int, c int, d int, e int, index idx(a))") - tk.MustExec("create table t2(a int, b int, c int)") - tk.MustExec("update t1 join t2 on t1.a=t2.a set t1.a=1 where t2.b=1 and t2.c=2") - - // Assign `DEFAULT` in `UPDATE` statement - tk.MustExec("drop table if exists t1, t2;") - tk.MustExec("create table t1 (a int default 1, b int default 2);") - tk.MustExec("insert into t1 values (10, 10), (20, 20);") - tk.MustExec("update t1 set a=default where b=10;") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 10", "20 20")) - tk.MustExec("update t1 set a=30, b=default where a=20;") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 10", "30 2")) - tk.MustExec("update t1 set a=default, b=default where a=30;") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 10", "1 2")) - tk.MustExec("insert into t1 values (40, 40)") - tk.MustExec("update t1 set a=default, b=default") - tk.MustQuery("select * from t1;").Check(testkit.Rows("1 2", "1 2", "1 2")) - tk.MustExec("update t1 set a=default(b), b=default(a)") - tk.MustQuery("select * from t1;").Check(testkit.Rows("2 1", "2 1", "2 1")) - // With generated columns - tk.MustExec("create table t2 (a int default 1, b int generated always as (-a) virtual, c int generated always as (-a) stored);") - tk.MustExec("insert into t2 values (10, default, default), (20, default, default)") - tk.MustExec("update t2 set b=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("10 -10 -10", "20 -20 -20")) - tk.MustExec("update t2 set a=30, b=default where a=10;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("30 -30 -30", "20 -20 -20")) - tk.MustExec("update t2 set c=default, a=40 where c=-20;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("30 -30 -30", "40 -40 -40")) - tk.MustExec("update t2 set a=default, b=default, c=default where b=-30;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 -1 -1", "40 -40 -40")) - tk.MustExec("update t2 set a=default(a), b=default, c=default;") - tk.MustQuery("select * from t2;").Check(testkit.Rows("1 -1 -1", "1 -1 -1")) - tk.MustGetErrCode("update t2 set b=default(a);", mysql.ErrBadGeneratedColumn) - tk.MustGetErrCode("update t2 set a=default(b), b=default(b);", mysql.ErrBadGeneratedColumn) - tk.MustGetErrCode("update t2 set a=default(a), c=default(c);", mysql.ErrBadGeneratedColumn) - tk.MustGetErrCode("update t2 set a=default(a), c=default(a);", mysql.ErrBadGeneratedColumn) - tk.MustExec("drop table t1, t2") -} - -func (s *testSuite4) TestPartitionedTableUpdate(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPartitionedTableUpdate(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 t") tk.MustExec(`create table t (id int not null default 1, name varchar(255)) @@ -1667,21 +1437,21 @@ func (s *testSuite4) TestPartitionedTableUpdate(c *C) { // update non partition column tk.MustExec(`UPDATE t SET name = "abc" where id > 0;`) tk.CheckExecResult(2, 0) - tk.CheckLastMessage("Rows matched: 2 Changed: 2 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") r := tk.MustQuery(`SELECT * from t order by id limit 2;`) r.Check(testkit.Rows("1 abc", "7 abc")) // update partition column tk.MustExec(`update t set id = id + 1`) tk.CheckExecResult(2, 0) - tk.CheckLastMessage("Rows matched: 2 Changed: 2 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") r = tk.MustQuery(`SELECT * from t order by id limit 2;`) r.Check(testkit.Rows("2 abc", "8 abc")) // update partition column, old and new record locates on different partitions tk.MustExec(`update t set id = 20 where id = 8`) tk.CheckExecResult(1, 0) - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") r = tk.MustQuery(`SELECT * from t order by id limit 2;`) r.Check(testkit.Rows("2 abc", "20 abc")) @@ -1696,14 +1466,13 @@ func (s *testSuite4) TestPartitionedTableUpdate(c *C) { tk.MustExec("insert into t(name) values ('aa')") tk.MustExec("update t set id = 8 where name = 'aa'") - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") tk.MustExec("insert into t(name) values ('bb')") r = tk.MustQuery("select * from t;") r.Check(testkit.Rows("8 aa", "9 bb")) _, err := tk.Exec("update t set id = null where name = 'aa'") - c.Assert(err, NotNil) - c.Assert(err.Error(), DeepEquals, "[table:1048]Column 'id' cannot be null") + require.EqualError(t, err, "[table:1048]Column 'id' cannot be null") // Test that in a transaction, when a constraint failed in an update statement, the record is not inserted. tk.MustExec("drop table if exists t;") @@ -1715,7 +1484,7 @@ func (s *testSuite4) TestPartitionedTableUpdate(c *C) { PARTITION p3 VALUES LESS THAN (21))`) tk.MustExec("insert t values (1, 1), (2, 2);") _, err = tk.Exec("update t set name = 1 where id = 2") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustQuery("select * from t").Check(testkit.Rows("1 1", "2 2")) // test update ignore for pimary key @@ -1727,15 +1496,15 @@ func (s *testSuite4) TestPartitionedTableUpdate(c *C) { tk.MustExec("insert into t values (5)") tk.MustExec("insert into t values (7)") _, err = tk.Exec("update ignore t set a = 5 where a = 7;") - c.Assert(err, IsNil) - tk.CheckLastMessage("Rows matched: 1 Changed: 0 Warnings: 1") + require.NoError(t, err) + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 1") r = tk.MustQuery("SHOW WARNINGS;") r.Check(testkit.Rows("Warning 1062 Duplicate entry '5' for key 'PRIMARY'")) tk.MustQuery("select * from t order by a").Check(testkit.Rows("5", "7")) // test update ignore for truncate as warning _, err = tk.Exec("update ignore t set a = 1 where a = (select '2a')") - c.Assert(err, IsNil) + require.NoError(t, err) r = tk.MustQuery("SHOW WARNINGS;") r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1292 Truncated incorrect DOUBLE value: '2a'")) @@ -1748,16 +1517,19 @@ func (s *testSuite4) TestPartitionedTableUpdate(c *C) { tk.MustExec("insert into t values (5)") tk.MustExec("insert into t values (7)") _, err = tk.Exec("update ignore t set a = 5 where a = 7;") - c.Assert(err, IsNil) - tk.CheckLastMessage("Rows matched: 1 Changed: 0 Warnings: 1") + require.NoError(t, err) + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 1") r = tk.MustQuery("SHOW WARNINGS;") r.Check(testkit.Rows("Warning 1062 Duplicate entry '5' for key 'I_uniq'")) tk.MustQuery("select * from t order by a").Check(testkit.Rows("5", "7")) } // TestUpdateCastOnlyModifiedValues for issue #4514. -func (s *testSuite4) TestUpdateCastOnlyModifiedValues(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUpdateCastOnlyModifiedValues(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 update_modified (col_1 int, col_2 enum('a', 'b'))") tk.MustExec("set SQL_MODE=''") @@ -1766,15 +1538,15 @@ func (s *testSuite4) TestUpdateCastOnlyModifiedValues(c *C) { r.Check(testkit.Rows("0 ")) tk.MustExec("set SQL_MODE=STRICT_ALL_TABLES") tk.MustExec("update update_modified set col_1 = 1") - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") r = tk.MustQuery("SELECT * FROM update_modified") r.Check(testkit.Rows("1 ")) _, err := tk.Exec("update update_modified set col_1 = 2, col_2 = 'c'") - c.Assert(err, NotNil) + require.Error(t, err) r = tk.MustQuery("SELECT * FROM update_modified") r.Check(testkit.Rows("1 ")) tk.MustExec("update update_modified set col_1 = 3, col_2 = 'a'") - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") r = tk.MustQuery("SELECT * FROM update_modified") r.Check(testkit.Rows("3 a")) @@ -1782,16 +1554,16 @@ func (s *testSuite4) TestUpdateCastOnlyModifiedValues(c *C) { tk.MustExec(`CREATE TABLE update_with_diff_type (a int, b JSON)`) tk.MustExec(`INSERT INTO update_with_diff_type VALUES(3, '{"a": "测试"}')`) tk.MustExec(`UPDATE update_with_diff_type SET a = '300'`) - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") r = tk.MustQuery("SELECT a FROM update_with_diff_type") r.Check(testkit.Rows("300")) tk.MustExec(`UPDATE update_with_diff_type SET b = '{"a": "\\u6d4b\\u8bd5"}'`) - tk.CheckLastMessage("Rows matched: 1 Changed: 0 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 0 Warnings: 0") r = tk.MustQuery("SELECT b FROM update_with_diff_type") r.Check(testkit.Rows(`{"a": "测试"}`)) } -func (s *testSuite4) fillMultiTableForUpdate(tk *testkit.TestKit) { +func fillMultiTableForUpdate(tk *testkit.TestKit) { // Create and fill table items tk.MustExec("CREATE TABLE items (id int, price TEXT);") tk.MustExec(`insert into items values (11, "items_price_11"), (12, "items_price_12"), (13, "items_price_13");`) @@ -1802,13 +1574,16 @@ func (s *testSuite4) fillMultiTableForUpdate(tk *testkit.TestKit) { tk.CheckExecResult(3, 0) } -func (s *testSuite4) TestMultipleTableUpdate(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMultipleTableUpdate(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - s.fillMultiTableForUpdate(tk) + fillMultiTableForUpdate(tk) tk.MustExec(`UPDATE items, month SET items.price=month.mprice WHERE items.id=month.mid;`) - tk.CheckLastMessage("Rows matched: 2 Changed: 2 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") tk.MustExec("begin") r := tk.MustQuery("SELECT * FROM items") r.Check(testkit.Rows("11 month_price_11", "12 items_price_12", "13 month_price_13")) @@ -1816,7 +1591,7 @@ func (s *testSuite4) TestMultipleTableUpdate(c *C) { // Single-table syntax but with multiple tables tk.MustExec(`UPDATE items join month on items.id=month.mid SET items.price=month.mid;`) - tk.CheckLastMessage("Rows matched: 2 Changed: 2 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") tk.MustExec("begin") r = tk.MustQuery("SELECT * FROM items") r.Check(testkit.Rows("11 11", "12 items_price_12", "13 13")) @@ -1824,7 +1599,7 @@ func (s *testSuite4) TestMultipleTableUpdate(c *C) { // JoinTable with alias table name. tk.MustExec(`UPDATE items T0 join month T1 on T0.id=T1.mid SET T0.price=T1.mprice;`) - tk.CheckLastMessage("Rows matched: 2 Changed: 2 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") tk.MustExec("begin") r = tk.MustQuery("SELECT * FROM items") r.Check(testkit.Rows("11 month_price_11", "12 items_price_12", "13 month_price_13")) @@ -1839,7 +1614,7 @@ func (s *testSuite4) TestMultipleTableUpdate(c *C) { insert into t2 values ("a"), ("b"); update t1, t2 set t1.c = 10, t2.c = "abc";` tk.MustExec(testSQL) - tk.CheckLastMessage("Rows matched: 4 Changed: 4 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 4 Changed: 4 Warnings: 0") // fix https://github.com/pingcap/tidb/issues/376 testSQL = `DROP TABLE IF EXISTS t1, t2; @@ -1849,7 +1624,7 @@ func (s *testSuite4) TestMultipleTableUpdate(c *C) { insert into t2 values (1), (2); update t1, t2 set t1.c1 = 10, t2.c2 = 2 where t2.c2 = 1;` tk.MustExec(testSQL) - tk.CheckLastMessage("Rows matched: 3 Changed: 3 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 3 Changed: 3 Warnings: 0") r = tk.MustQuery("select * from t1") r.Check(testkit.Rows("10", "10")) @@ -1858,18 +1633,21 @@ func (s *testSuite4) TestMultipleTableUpdate(c *C) { tk.MustExec("drop table if exists t, t") tk.MustExec("create table t (a int, b int)") tk.MustExec("insert into t values(1, 1), (2, 2), (3, 3)") - tk.CheckLastMessage("Records: 3 Duplicates: 0 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Records: 3 Duplicates: 0 Warnings: 0") tk.MustExec("update t m, t n set m.a = m.a + 1") - tk.CheckLastMessage("Rows matched: 3 Changed: 3 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 3 Changed: 3 Warnings: 0") tk.MustQuery("select * from t").Check(testkit.Rows("2 1", "3 2", "4 3")) tk.MustExec("update t m, t n set n.a = n.a - 1, n.b = n.b + 1") - tk.CheckLastMessage("Rows matched: 3 Changed: 3 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 3 Changed: 3 Warnings: 0") tk.MustQuery("select * from t").Check(testkit.Rows("1 2", "2 3", "3 4")) } -func (s *testSuite) TestDelete(c *C) { - tk := testkit.NewTestKit(c, s.store) - s.fillData(tk, "delete_test") +func TestDelete(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + fillData(tk, "delete_test") tk.MustExec(`update delete_test set name = "abc" where id = 2;`) tk.CheckExecResult(1, 0) @@ -1894,9 +1672,9 @@ func (s *testSuite) TestDelete(c *C) { // Test delete ignore tk.MustExec("insert into delete_test values (2, 'abc')") _, err := tk.Exec("delete from delete_test where id = (select '2a')") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("delete ignore from delete_test where id = (select '2a')") - c.Assert(err, IsNil) + require.NoError(t, err) tk.CheckExecResult(1, 0) r := tk.MustQuery("SHOW WARNINGS;") r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1292 Truncated incorrect DOUBLE value: '2a'")) @@ -1906,16 +1684,16 @@ func (s *testSuite) TestDelete(c *C) { tk.MustExec("create view v as select * from delete_test") _, err = tk.Exec("delete from v where name = 'aaa'") - c.Assert(err.Error(), Equals, core.ErrViewInvalid.GenWithStackByArgs("test", "v").Error()) + require.EqualError(t, err, core.ErrViewInvalid.GenWithStackByArgs("test", "v").Error()) tk.MustExec("drop view v") tk.MustExec("create sequence seq") _, err = tk.Exec("delete from seq") - c.Assert(err.Error(), Equals, "delete sequence seq is not supported now.") + require.EqualError(t, err, "delete sequence seq is not supported now.") tk.MustExec("drop sequence seq") } -func (s *testSuite4) TestPartitionedTableDelete(c *C) { +func TestPartitionedTableDelete(t *testing.T) { createTable := `CREATE TABLE test.t (id int not null default 1, name varchar(255), index(id)) PARTITION BY RANGE ( id ) ( PARTITION p0 VALUES LESS THAN (6), @@ -1923,7 +1701,10 @@ func (s *testSuite4) TestPartitionedTableDelete(c *C) { PARTITION p2 VALUES LESS THAN (16), PARTITION p3 VALUES LESS THAN (21))` - tk := testkit.NewTestKit(c, s.store) + t.Parallel() + 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(createTable) @@ -1945,9 +1726,9 @@ func (s *testSuite4) TestPartitionedTableDelete(c *C) { // Test delete ignore tk.MustExec("insert into t values (2, 'abc')") _, err := tk.Exec("delete from t where id = (select '2a')") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("delete ignore from t where id = (select '2a')") - c.Assert(err, IsNil) + require.NoError(t, err) tk.CheckExecResult(1, 0) r := tk.MustQuery("SHOW WARNINGS;") r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '2a'", "Warning 1292 Truncated incorrect DOUBLE value: '2a'")) @@ -1968,7 +1749,7 @@ func (s *testSuite4) TestPartitionedTableDelete(c *C) { tk.MustExec(`drop table t1;`) } -func (s *testSuite4) fillDataMultiTable(tk *testkit.TestKit) { +func fillDataMultiTable(tk *testkit.TestKit) { tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2, t3") // Create and fill table t1 @@ -1985,20 +1766,26 @@ func (s *testSuite4) fillDataMultiTable(tk *testkit.TestKit) { tk.CheckExecResult(3, 0) } -func (s *testSuite4) TestMultiTableDelete(c *C) { - tk := testkit.NewTestKit(c, s.store) - s.fillDataMultiTable(tk) +func TestMultiTableDelete(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + fillDataMultiTable(tk) tk.MustExec(`delete t1, t2 from t1 inner join t2 inner join t3 where t1.id=t2.id and t2.id=t3.id;`) tk.CheckExecResult(2, 0) // Select data r := tk.MustQuery("select * from t3") - c.Assert(r.Rows(), HasLen, 3) + require.Len(t, r.Rows(), 3) } -func (s *testSuite4) TestQualifiedDelete(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestQualifiedDelete(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 t1") tk.MustExec("drop table if exists t2") @@ -2015,8 +1802,7 @@ func (s *testSuite4) TestQualifiedDelete(c *C) { tk.CheckExecResult(1, 0) r := tk.MustQuery("select * from t1") - c.Assert(r.Rows(), HasLen, 0) - + require.Len(t, r.Rows(), 0) tk.MustExec("insert into t1 values (1, 3)") tk.MustExec("delete from t1 as a where a.c1 = 1") tk.CheckExecResult(1, 0) @@ -2031,26 +1817,29 @@ func (s *testSuite4) TestQualifiedDelete(c *C) { tk.CheckExecResult(2, 0) _, err := tk.Exec("delete t1, t2 from t1 as a join t2 as b where a.c2 = b.c1") - c.Assert(err, NotNil) + require.Error(t, err) } -func (s *testSuite8) TestLoadDataMissingColumn(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestLoadDataMissingColumn(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") createSQL := `create table load_data_missing (id int, t timestamp not null)` tk.MustExec(createSQL) tk.MustExec("load data local infile '/tmp/nonexistence.csv' ignore into table load_data_missing") - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session().(sessionctx.Context) ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) - c.Assert(ok, IsTrue) + require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) - c.Assert(ld, NotNil) + require.NotNil(t, ld) deleteSQL := "delete from load_data_missing" selectSQL := "select id, hour(t), minute(t) from load_data_missing;" _, reachLimit, err := ld.InsertData(context.Background(), nil, nil) - c.Assert(err, IsNil) - c.Assert(reachLimit, IsFalse) + require.NoError(t, err) + require.False(t, reachLimit) r := tk.MustQuery(selectSQL) r.Check(nil) @@ -2060,7 +1849,7 @@ func (s *testSuite8) TestLoadDataMissingColumn(c *C) { tests := []testCase{ {nil, []byte("12\n"), []string{fmt.Sprintf("12|%v|%v", timeHour, timeMinute)}, nil, "Records: 1 Deleted: 0 Skipped: 0 Warnings: 0"}, } - checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) + checkCases(tests, ld, t, tk, ctx, selectSQL, deleteSQL) tk.MustExec("alter table load_data_missing add column t2 timestamp null") curTime = types.CurrentTime(mysql.TypeTimestamp) @@ -2070,22 +1859,24 @@ func (s *testSuite8) TestLoadDataMissingColumn(c *C) { tests = []testCase{ {nil, []byte("12\n"), []string{fmt.Sprintf("12|%v|%v|", timeHour, timeMinute)}, nil, "Records: 1 Deleted: 0 Skipped: 0 Warnings: 0"}, } - checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) - + checkCases(tests, ld, t, tk, ctx, selectSQL, deleteSQL) } -func (s *testSuite4) TestIssue18681(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue18681(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") createSQL := `drop table if exists load_data_test; create table load_data_test (a bit(1),b bit(1),c bit(1),d bit(1));` tk.MustExec(createSQL) tk.MustExec("load data local infile '/tmp/nonexistence.csv' ignore into table load_data_test") - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session().(sessionctx.Context) ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) - c.Assert(ok, IsTrue) + require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) - c.Assert(ld, NotNil) + require.NotNil(t, ld) deleteSQL := "delete from load_data_test" selectSQL := "select bin(a), bin(b), bin(c), bin(d) from load_data_test;" @@ -2102,29 +1893,32 @@ func (s *testSuite4) TestIssue18681(c *C) { tests := []testCase{ {nil, []byte("true\tfalse\t0\t1\n"), []string{"1|0|0|1"}, nil, "Records: 1 Deleted: 0 Skipped: 0 Warnings: 0"}, } - checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) - c.Assert(sc.WarningCount(), Equals, uint16(0)) + checkCases(tests, ld, t, tk, ctx, selectSQL, deleteSQL) + require.Equal(t, uint16(0), sc.WarningCount()) } -func (s *testSuite4) TestLoadData(c *C) { +func TestLoadData(t *testing.T) { trivialMsg := "Records: 1 Deleted: 0 Skipped: 0 Warnings: 0" - tk := testkit.NewTestKit(c, s.store) + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") createSQL := `drop table if exists load_data_test; create table load_data_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 varchar(255) default "def", c3 int);` _, err := tk.Exec("load data local infile '/tmp/nonexistence.csv' into table load_data_test") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec(createSQL) _, err = tk.Exec("load data infile '/tmp/nonexistence.csv' into table load_data_test") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("load data local infile '/tmp/nonexistence.csv' replace into table load_data_test") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("load data local infile '/tmp/nonexistence.csv' ignore into table load_data_test") - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session().(sessionctx.Context) ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) - c.Assert(ok, IsTrue) + require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) - c.Assert(ld, NotNil) + require.NotNil(t, ld) deleteSQL := "delete from load_data_test" selectSQL := "select * from load_data_test;" @@ -2132,10 +1926,10 @@ func (s *testSuite4) TestLoadData(c *C) { ctx.GetSessionVars().StmtCtx.DupKeyAsWarning = true ctx.GetSessionVars().StmtCtx.BadNullAsWarning = true _, reachLimit, err := ld.InsertData(context.Background(), nil, nil) - c.Assert(err, IsNil) - c.Assert(reachLimit, IsFalse) + require.NoError(t, err) + require.False(t, reachLimit) err = ld.CheckAndInsertOneBatch(context.Background(), ld.GetRows(), ld.GetCurBatchCnt()) - c.Assert(err, IsNil) + require.NoError(t, err) ld.SetMaxRowsInBatch(20000) r := tk.MustQuery(selectSQL) r.Check(nil) @@ -2169,7 +1963,7 @@ func (s *testSuite4) TestLoadData(c *C) { // data1 != nil, data2 != nil, InsertData returns data isn't nil {[]byte("\t2\t3"), []byte("\t4\t5"), nil, []byte("\t2\t3\t4\t5"), "Records: 0 Deleted: 0 Skipped: 0 Warnings: 0"}, } - checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) + checkCases(tests, ld, t, tk, ctx, selectSQL, deleteSQL) // lines starting symbol is "" and terminated symbol length is 2, InsertData returns data is nil ld.LinesInfo.Terminated = "||" @@ -2185,7 +1979,7 @@ func (s *testSuite4) TestLoadData(c *C) { []string{"4|2|3|4", "5|22|33|", "6|222||"}, nil, "Records: 3 Deleted: 0 Skipped: 0 Warnings: 0"}, {[]byte("6\t2\t3"), []byte("4\t5||"), []string{"6|2|34|5"}, nil, trivialMsg}, } - checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) + checkCases(tests, ld, t, tk, ctx, selectSQL, deleteSQL) // fields and lines aren't default, InsertData returns data is nil ld.FieldsInfo.Terminated = "\\" @@ -2238,7 +2032,7 @@ func (s *testSuite4) TestLoadData(c *C) { []string{"1|2|3|4", "2|22|33|"}, []byte("#^"), "Records: 2 Deleted: 0 Skipped: 0 Warnings: 0"}, {[]byte("xx1\\2\\3"), []byte("\\4\\5|!#^"), nil, []byte("#^"), "Records: 0 Deleted: 0 Skipped: 0 Warnings: 0"}, } - checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) + checkCases(tests, ld, t, tk, ctx, selectSQL, deleteSQL) // lines starting symbol is the same as terminated symbol, InsertData returns data is nil ld.LinesInfo.Terminated = "xxx" @@ -2273,7 +2067,7 @@ func (s *testSuite4) TestLoadData(c *C) { {[]byte("xxx10\\2\\3\\4\\5x"), []byte("xx11\\22\\33xxxxxx12\\222xxx"), []string{"10|2|3|4", "40|||"}, []byte("xxx"), "Records: 2 Deleted: 0 Skipped: 0 Warnings: 1"}, } - checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) + checkCases(tests, ld, t, tk, ctx, selectSQL, deleteSQL) // test line terminator in field quoter ld.LinesInfo.Terminated = "\n" @@ -2281,7 +2075,7 @@ func (s *testSuite4) TestLoadData(c *C) { tests = []testCase{ {[]byte("xxx1\\1\\\"2\n\"\\3\nxxx4\\4\\\"5\n5\"\\6"), nil, []string{"1|1|2\n|3", "4|4|5\n5|6"}, nil, "Records: 2 Deleted: 0 Skipped: 0 Warnings: 0"}, } - checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) + checkCases(tests, ld, t, tk, ctx, selectSQL, deleteSQL) ld.LinesInfo.Terminated = "#\n" ld.FieldsInfo.Terminated = "#" @@ -2290,7 +2084,7 @@ func (s *testSuite4) TestLoadData(c *C) { {[]byte("xxx1#2#3#4#\nnxxx2#3#4#5#\n"), nil, []string{"1|2|3|4", "2|3|4|5"}, nil, "Records: 2 Deleted: 0 Skipped: 0 Warnings: 0"}, {[]byte("xxx1#2#\"3#\"#\"4\n\"#\nxxx2#3#\"#4#\n\"#5#\n"), nil, []string{"1|2|3#|4", "2|3|#4#\n|5"}, nil, "Records: 2 Deleted: 0 Skipped: 0 Warnings: 0"}, } - checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) + checkCases(tests, ld, t, tk, ctx, selectSQL, deleteSQL) ld.LinesInfo.Terminated = "#" ld.FieldsInfo.Terminated = "##" @@ -2300,20 +2094,23 @@ func (s *testSuite4) TestLoadData(c *C) { {[]byte("1##2##3##4#2##3##4##5#"), nil, []string{"1|2|3|4", "2|3|4|5"}, nil, "Records: 2 Deleted: 0 Skipped: 0 Warnings: 0"}, {[]byte("1##2##\"3##\"##\"4\n\"#2##3##\"##4#\"##5#"), nil, []string{"1|2|3##|4", "2|3|##4#|5"}, nil, "Records: 2 Deleted: 0 Skipped: 0 Warnings: 0"}, } - checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) + checkCases(tests, ld, t, tk, ctx, selectSQL, deleteSQL) } -func (s *testSuite4) TestLoadDataEscape(c *C) { +func TestLoadDataEscape(t *testing.T) { trivialMsg := "Records: 1 Deleted: 0 Skipped: 0 Warnings: 0" - tk := testkit.NewTestKit(c, s.store) + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test; drop table if exists load_data_test;") tk.MustExec("CREATE TABLE load_data_test (id INT NOT NULL PRIMARY KEY, value TEXT NOT NULL) CHARACTER SET utf8") tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table load_data_test") - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session().(sessionctx.Context) ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) - c.Assert(ok, IsTrue) + require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) - c.Assert(ld, NotNil) + require.NotNil(t, ld) // test escape tests := []testCase{ // data1 = nil, data2 != nil @@ -2329,21 +2126,24 @@ func (s *testSuite4) TestLoadDataEscape(c *C) { } deleteSQL := "delete from load_data_test" selectSQL := "select * from load_data_test;" - checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) + checkCases(tests, ld, t, tk, ctx, selectSQL, deleteSQL) } // TestLoadDataSpecifiedColumns reuse TestLoadDataEscape's test case :-) -func (s *testSuite4) TestLoadDataSpecifiedColumns(c *C) { +func TestLoadDataSpecifiedColumns(t *testing.T) { trivialMsg := "Records: 1 Deleted: 0 Skipped: 0 Warnings: 0" - tk := testkit.NewTestKit(c, s.store) + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test; drop table if exists load_data_test;") tk.MustExec(`create table load_data_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 varchar(255) default "def", c3 int default 0);`) tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table load_data_test (c1, c2)") - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session().(sessionctx.Context) ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) - c.Assert(ok, IsTrue) + require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) - c.Assert(ld, NotNil) + require.NotNil(t, ld) // test tests := []testCase{ // data1 = nil, data2 != nil @@ -2357,75 +2157,87 @@ func (s *testSuite4) TestLoadDataSpecifiedColumns(c *C) { } deleteSQL := "delete from load_data_test" selectSQL := "select * from load_data_test;" - checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) + checkCases(tests, ld, t, tk, ctx, selectSQL, deleteSQL) } -func (s *testSuite4) TestLoadDataIgnoreLines(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestLoadDataIgnoreLines(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test; drop table if exists load_data_test;") tk.MustExec("CREATE TABLE load_data_test (id INT NOT NULL PRIMARY KEY, value TEXT NOT NULL) CHARACTER SET utf8") tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table load_data_test ignore 1 lines") - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session().(sessionctx.Context) ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) - c.Assert(ok, IsTrue) + require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) - c.Assert(ld, NotNil) + require.NotNil(t, ld) tests := []testCase{ {nil, []byte("1\tline1\n2\tline2\n"), []string{"2|line2"}, nil, "Records: 1 Deleted: 0 Skipped: 0 Warnings: 0"}, {nil, []byte("1\tline1\n2\tline2\n3\tline3\n"), []string{"2|line2", "3|line3"}, nil, "Records: 2 Deleted: 0 Skipped: 0 Warnings: 0"}, } deleteSQL := "delete from load_data_test" selectSQL := "select * from load_data_test;" - checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) + checkCases(tests, ld, t, tk, ctx, selectSQL, deleteSQL) } // TestLoadDataOverflowBigintUnsigned related to issue 6360 -func (s *testSuite4) TestLoadDataOverflowBigintUnsigned(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestLoadDataOverflowBigintUnsigned(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test; drop table if exists load_data_test;") tk.MustExec("CREATE TABLE load_data_test (a bigint unsigned);") tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table load_data_test") - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session().(sessionctx.Context) ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) - c.Assert(ok, IsTrue) + require.True(t, ok) defer ctx.SetValue(executor.LoadDataVarKey, nil) - c.Assert(ld, NotNil) + require.NotNil(t, ld) tests := []testCase{ {nil, []byte("-1\n-18446744073709551615\n-18446744073709551616\n"), []string{"0", "0", "0"}, nil, "Records: 3 Deleted: 0 Skipped: 0 Warnings: 3"}, {nil, []byte("-9223372036854775809\n18446744073709551616\n"), []string{"0", "18446744073709551615"}, nil, "Records: 2 Deleted: 0 Skipped: 0 Warnings: 2"}, } deleteSQL := "delete from load_data_test" selectSQL := "select * from load_data_test;" - checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) + checkCases(tests, ld, t, tk, ctx, selectSQL, deleteSQL) } -func (s *testSuite4) TestLoadDataIntoPartitionedTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestLoadDataIntoPartitionedTable(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 range_t (a int, b int) partition by range (a) ( " + "partition p0 values less than (4)," + "partition p1 values less than (7)," + "partition p2 values less than (11))") tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table range_t fields terminated by ','") - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session().(sessionctx.Context) ld := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) - c.Assert(ctx.NewTxn(context.Background()), IsNil) + require.Nil(t, ctx.NewTxn(context.Background())) _, _, err := ld.InsertData(context.Background(), nil, []byte("1,2\n3,4\n5,6\n7,8\n9,10\n")) - c.Assert(err, IsNil) + require.NoError(t, err) err = ld.CheckAndInsertOneBatch(context.Background(), ld.GetRows(), ld.GetCurBatchCnt()) - c.Assert(err, IsNil) + require.NoError(t, err) ld.SetMaxRowsInBatch(20000) ld.SetMessage() ctx.StmtCommit() txn, err := ctx.Txn(true) - c.Assert(err, IsNil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testSuite4) TestNullDefault(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestNullDefault(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test; drop table if exists test_null_default;") tk.MustExec("set timestamp = 1234") tk.MustExec("set time_zone = '+08:00'") @@ -2436,8 +2248,11 @@ func (s *testSuite4) TestNullDefault(c *C) { tk.MustQuery("select * from test_null_default").Check(testkit.Rows("", "1970-01-01 08:20:34")) } -func (s *testSuite4) TestNotNullDefault(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestNotNullDefault(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test; drop table if exists t1,t2;") defer tk.MustExec("drop table t1,t2") tk.MustExec("create table t1 (a int not null default null default 1);") @@ -2445,28 +2260,28 @@ func (s *testSuite4) TestNotNullDefault(c *C) { tk.MustExec("alter table t2 change column a a int not null default null default 1;") } -func (s *testBypassSuite) TestLatch(c *C) { +func TestLatch(t *testing.T) { store, err := mockstore.NewMockStore( // Small latch slot size to make conflicts. mockstore.WithTxnLocalLatches(64), ) - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := store.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() dom, err1 := session.BootstrapSession(store) - c.Assert(err1, IsNil) + require.Nil(t, err1) defer dom.Close() - tk1 := testkit.NewTestKit(c, store) + tk1 := testkit.NewTestKit(t, store) tk1.MustExec("use test") tk1.MustExec("drop table if exists t") tk1.MustExec("create table t (id int)") tk1.MustExec("set @@tidb_disable_txn_auto_retry = true") - tk2 := testkit.NewTestKit(c, store) + tk2 := testkit.NewTestKit(t, store) tk2.MustExec("use test") tk1.MustExec("set @@tidb_disable_txn_auto_retry = true") @@ -2496,7 +2311,7 @@ func (s *testBypassSuite) TestLatch(c *C) { tk1.MustExec("update t set id = id + 1") tk2.MustExec("update t set id = id + 1") _, err = tk1.Exec("commit") - c.Assert(kv.ErrWriteConflictInTiDB.Equal(err), IsTrue) + require.True(t, kv.ErrWriteConflictInTiDB.Equal(err)) tk1.MustExec("set @@tidb_disable_txn_auto_retry = 0") tk1.MustExec("update t set id = id + 1") @@ -2505,8 +2320,11 @@ func (s *testBypassSuite) TestLatch(c *C) { } // TestIssue4067 Test issue https://github.com/pingcap/tidb/issues/4067 -func (s *testSuite7) TestIssue4067(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue4067(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 t1, t2") tk.MustExec("create table t1(id int)") @@ -2519,8 +2337,11 @@ func (s *testSuite7) TestIssue4067(c *C) { tk.MustQuery("select * from t1").Check(nil) } -func (s *testSuite7) TestInsertCalculatedValue(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertCalculatedValue(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 t") @@ -2635,44 +2456,53 @@ func (s *testSuite7) TestInsertCalculatedValue(c *C) { tk.MustQuery("select * from t").Check(testkit.Rows("4 0 2")) } -func (s *testSuite7) TestDataTooLongErrMsg(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDataTooLongErrMsg(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 varchar(2));") _, err := tk.Exec("insert into t values('123');") - c.Assert(types.ErrDataTooLong.Equal(err), IsTrue) - c.Assert(err.Error(), Equals, "[types:1406]Data too long for column 'a' at row 1") + require.True(t, types.ErrDataTooLong.Equal(err)) + require.EqualError(t, err, "[types:1406]Data too long for column 'a' at row 1") tk.MustExec("insert into t values('12')") _, err = tk.Exec("update t set a = '123' where a = '12';") - c.Assert(types.ErrDataTooLong.Equal(err), IsTrue) - c.Assert(err.Error(), Equals, "[types:1406]Data too long for column 'a' at row 1") + require.True(t, types.ErrDataTooLong.Equal(err)) + require.EqualError(t, err, "[types:1406]Data too long for column 'a' at row 1") } -func (s *testSuite7) TestUpdateSelect(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUpdateSelect(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 msg (id varchar(8), b int, status int, primary key (id, b))") tk.MustExec("insert msg values ('abc', 1, 1)") tk.MustExec("create table detail (id varchar(8), start varchar(8), status int, index idx_start(start))") tk.MustExec("insert detail values ('abc', '123', 2)") tk.MustExec("UPDATE msg SET msg.status = (SELECT detail.status FROM detail WHERE msg.id = detail.id)") - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") tk.MustExec("admin check table msg") } -func (s *testSuite7) TestUpdateDelete(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUpdateDelete(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 ttt (id bigint(20) NOT NULL, host varchar(30) NOT NULL, PRIMARY KEY (id), UNIQUE KEY i_host (host));") tk.MustExec("insert into ttt values (8,8),(9,9);") tk.MustExec("begin") tk.MustExec("update ttt set id = 0, host='9' where id = 9 limit 1;") - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") tk.MustExec("delete from ttt where id = 0 limit 1;") tk.MustQuery("select * from ttt use index (i_host) order by host;").Check(testkit.Rows("8 8")) tk.MustExec("update ttt set id = 0, host='8' where id = 8 limit 1;") - tk.CheckLastMessage("Rows matched: 1 Changed: 1 Warnings: 0") + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 1 Changed: 1 Warnings: 0") tk.MustExec("delete from ttt where id = 0 limit 1;") tk.MustQuery("select * from ttt use index (i_host) order by host;").Check(testkit.Rows()) tk.MustExec("commit") @@ -2680,78 +2510,85 @@ func (s *testSuite7) TestUpdateDelete(c *C) { tk.MustExec("drop table ttt") } -func (s *testSuite7) TestUpdateAffectRowCnt(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUpdateAffectRowCnt(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 a(id int auto_increment, a int default null, primary key(id))") tk.MustExec("insert into a values (1, 1001), (2, 1001), (10001, 1), (3, 1)") tk.MustExec("update a set id = id*10 where a = 1001") - ctx := tk.Se.(sessionctx.Context) - c.Assert(ctx.GetSessionVars().StmtCtx.AffectedRows(), Equals, uint64(2)) - tk.CheckLastMessage("Rows matched: 2 Changed: 2 Warnings: 0") + ctx := tk.Session().(sessionctx.Context) + require.Equal(t, uint64(2), ctx.GetSessionVars().StmtCtx.AffectedRows()) + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") tk.MustExec("drop table a") tk.MustExec("create table a ( a bigint, b bigint)") tk.MustExec("insert into a values (1, 1001), (2, 1001), (10001, 1), (3, 1)") tk.MustExec("update a set a = a*10 where b = 1001") - ctx = tk.Se.(sessionctx.Context) - c.Assert(ctx.GetSessionVars().StmtCtx.AffectedRows(), Equals, uint64(2)) - tk.CheckLastMessage("Rows matched: 2 Changed: 2 Warnings: 0") + ctx = tk.Session().(sessionctx.Context) + require.Equal(t, uint64(2), ctx.GetSessionVars().StmtCtx.AffectedRows()) + require.Equal(t, tk.Session().LastMessage(), "Rows matched: 2 Changed: 2 Warnings: 0") } -func (s *testSuite7) TestReplaceLog(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestReplaceLog(t *testing.T) { + t.Parallel() + store, domain, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table testLog (a int not null primary key, b int unique key);`) // 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("testLog") tbl, err := is.TableByName(dbName, tblName) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := tbl.Meta() idxInfo := tblInfo.FindIndexByName("b") indexOpr := tables.NewIndex(tblInfo.ID, tblInfo, idxInfo) - txn, err := s.store.Begin() - c.Assert(err, IsNil) - _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(1), kv.IntHandle(1), nil) - c.Assert(err, IsNil) + txn, err := store.Begin() + require.NoError(t, err) + _, err = indexOpr.Create(ctx, txn, types.MakeDatums(1), kv.IntHandle(1), nil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = tk.Exec(`replace into testLog values (0, 0), (1, 1);`) - c.Assert(err, NotNil) - expErr := errors.New(`can not be duplicated row, due to old row not found. handle 1 not found`) - c.Assert(expErr.Error() == err.Error(), IsTrue, Commentf("obtained error: (%s)\nexpected error: (%s)", err.Error(), expErr.Error())) - + require.Error(t, err) + require.EqualError(t, err, `can not be duplicated row, due to old row not found. handle 1 not found`) tk.MustQuery(`admin cleanup index testLog b;`).Check(testkit.Rows("1")) } // TestRebaseIfNeeded is for issue 7422. // There is no need to do the rebase when updating a record if the auto-increment ID not changed. // This could make the auto ID increasing speed slower. -func (s *testSuite7) TestRebaseIfNeeded(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestRebaseIfNeeded(t *testing.T) { + t.Parallel() + store, domain, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table t (a int not null primary key auto_increment, b int unique key);`) tk.MustExec(`insert into t (b) values (1);`) - s.ctx = mock.NewContext() - s.ctx.Store = s.store - tbl, err := s.domain.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - c.Assert(s.ctx.NewTxn(context.Background()), IsNil) + ctx := mock.NewContext() + ctx.Store = store + tbl, err := domain.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + require.Nil(t, ctx.NewTxn(context.Background())) // AddRecord directly here will skip to rebase the auto ID in the insert statement, // which could simulate another TiDB adds a large auto ID. - _, err = tbl.AddRecord(s.ctx, types.MakeDatums(30001, 2)) - c.Assert(err, IsNil) - txn, err := s.ctx.Txn(true) - c.Assert(err, IsNil) - c.Assert(txn.Commit(context.Background()), IsNil) + _, err = tbl.AddRecord(ctx, types.MakeDatums(30001, 2)) + require.NoError(t, err) + txn, err := ctx.Txn(true) + require.NoError(t, err) + require.Nil(t, txn.Commit(context.Background())) tk.MustExec(`update t set b = 3 where a = 30001;`) tk.MustExec(`insert into t (b) values (4);`) @@ -2766,8 +2603,11 @@ func (s *testSuite7) TestRebaseIfNeeded(c *C) { tk.MustQuery(`select a from t where b = 6;`).Check(testkit.Rows("30003")) } -func (s *testSuite7) TestDeferConstraintCheckForDelete(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDeferConstraintCheckForDelete(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set tidb_constraint_check_in_place = 0") tk.MustExec("set @@tidb_txn_mode = 'optimistic'") tk.MustExec("use test") @@ -2779,7 +2619,7 @@ func (s *testSuite7) TestDeferConstraintCheckForDelete(c *C) { tk.MustExec("insert into t1 values(1, 3)") tk.MustExec("delete from t1 where j = 3") _, err := tk.Exec("commit") - c.Assert(err.Error(), Equals, "previous statement: delete from t1 where j = 3: [kv:1062]Duplicate entry '1' for key 'PRIMARY'") + require.EqualError(t, err, "previous statement: delete from t1 where j = 3: [kv:1062]Duplicate entry '1' for key 'PRIMARY'") tk.MustExec("rollback") tk.MustExec("create table t2(i int, j int, unique index idx(i))") @@ -2788,7 +2628,7 @@ func (s *testSuite7) TestDeferConstraintCheckForDelete(c *C) { tk.MustExec("insert into t2 values(1, 3)") tk.MustExec("delete from t2 where j = 3") _, err = tk.Exec("commit") - c.Assert(err.Error(), Equals, "previous statement: delete from t2 where j = 3: [kv:1062]Duplicate entry '1' for key 'idx'") + require.EqualError(t, err, "previous statement: delete from t2 where j = 3: [kv:1062]Duplicate entry '1' for key 'idx'") tk.MustExec("admin check table t2") tk.MustExec("create table t3(i int, j int, primary key(i))") @@ -2816,21 +2656,24 @@ func (s *testSuite7) TestDeferConstraintCheckForDelete(c *C) { tk.MustQuery("select * from t5").Check(testkit.Rows("1 4")) } -func (s *testSuite7) TestDeferConstraintCheckForInsert(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDeferConstraintCheckForInsert(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 t;create table t (a int primary key, b int);`) tk.MustExec(`insert into t values (1,2),(2,2)`) _, err := tk.Exec("update t set a=a+1 where b=2") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec(`drop table if exists t;create table t (i int key);`) tk.MustExec(`insert t values (1);`) tk.MustExec(`set tidb_constraint_check_in_place = 1;`) tk.MustExec(`begin;`) _, err = tk.Exec(`insert t values (1);`) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec(`update t set i = 2 where i = 1;`) tk.MustExec(`commit;`) tk.MustQuery(`select * from t;`).Check(testkit.Rows("2")) @@ -2839,9 +2682,9 @@ func (s *testSuite7) TestDeferConstraintCheckForInsert(c *C) { tk.MustExec("replace into t values (1),(2)") tk.MustExec("begin") _, err = tk.Exec("update t set i = 2 where i = 1") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("insert into t values (1) on duplicate key update i = i + 1") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") tk.MustExec(`drop table t; create table t (id int primary key, v int unique);`) @@ -2850,16 +2693,16 @@ func (s *testSuite7) TestDeferConstraintCheckForInsert(c *C) { tk.MustExec(`set @@autocommit = 0;`) _, err = tk.Exec("insert into t values (3, 1)") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("insert into t values (1, 3)") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("commit") tk.MustExec(`set tidb_constraint_check_in_place = 0;`) tk.MustExec("insert into t values (3, 1)") tk.MustExec("insert into t values (1, 3)") _, err = tk.Exec("commit") - c.Assert(err, NotNil) + require.Error(t, err) // Cover the temporary table. for val := range []int{0, 1} { @@ -2870,12 +2713,12 @@ func (s *testSuite7) TestDeferConstraintCheckForInsert(c *C) { tk.MustExec("begin") tk.MustExec("insert into t values (1, 1)") _, err = tk.Exec(`insert into t values (1, 3)`) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("insert into t values (2, 2)") _, err = tk.Exec("update t set a = a + 1 where a = 1") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("insert into t values (1, 3) on duplicated key update a = a + 1") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("commit") tk.MustExec("drop table t") @@ -2883,12 +2726,12 @@ func (s *testSuite7) TestDeferConstraintCheckForInsert(c *C) { tk.MustExec("begin") tk.MustExec("insert into t values (1, 1)") _, err = tk.Exec(`insert into t values (3, 1)`) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("insert into t values (2, 2)") _, err = tk.Exec("update t set b = b + 1 where a = 1") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("insert into t values (3, 1) on duplicated key update b = b + 1") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("commit") // cases for temporary table @@ -2897,22 +2740,22 @@ func (s *testSuite7) TestDeferConstraintCheckForInsert(c *C) { tk.MustExec("begin") tk.MustExec("insert into tl values (1, 1)") _, err = tk.Exec(`insert into tl values (1, 3)`) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("insert into tl values (2, 2)") _, err = tk.Exec("update tl set a = a + 1 where a = 1") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("insert into tl values (1, 3) on duplicated key update a = a + 1") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("commit") tk.MustExec("begin") tk.MustQuery("select * from tl").Check(testkit.Rows("1 1", "2 2")) _, err = tk.Exec(`insert into tl values (1, 3)`) - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("update tl set a = a + 1 where a = 1") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("insert into tl values (1, 3) on duplicated key update a = a + 1") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") tk.MustExec("drop table tl") @@ -2920,29 +2763,35 @@ func (s *testSuite7) TestDeferConstraintCheckForInsert(c *C) { tk.MustExec("begin") tk.MustExec("insert into tl values (1, 1)") _, err = tk.Exec(`insert into tl values (3, 1)`) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("insert into tl values (2, 2)") _, err = tk.Exec("update tl set b = b + 1 where a = 1") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("insert into tl values (3, 1) on duplicated key update b = b + 1") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("commit") tk.MustExec("begin") tk.MustQuery("select * from tl").Check(testkit.Rows("1 1", "2 2")) _, err = tk.Exec(`insert into tl values (3, 1)`) - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("update tl set b = b + 1 where a = 1") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("insert into tl values (3, 1) on duplicated key update b = b + 1") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("rollback") } } -func (s *testSuite7) TestPessimisticDeleteYourWrites(c *C) { - session1 := testkit.NewTestKitWithInit(c, s.store) - session2 := testkit.NewTestKitWithInit(c, s.store) +func TestPessimisticDeleteYourWrites(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + + session1 := testkit.NewTestKit(t, store) + session1.MustExec("use test") + session2 := testkit.NewTestKit(t, store) + session2.MustExec("use test") session1.MustExec("drop table if exists x;") session1.MustExec("create table x (id int primary key, c int);") @@ -2954,37 +2803,45 @@ func (s *testSuite7) TestPessimisticDeleteYourWrites(c *C) { session1.MustExec("insert into x select 1, 1") session1.MustExec("delete from x where id = 1") session2.MustExec("begin;") - var wg sync.WaitGroup - wg.Add(1) - go func() { + var wg util.WaitGroupWrapper + wg.Run(func() { session2.MustExec("insert into x select 1, 2") - wg.Done() - }() + }) session1.MustExec("commit;") wg.Wait() session2.MustExec("commit;") session2.MustQuery("select * from x").Check(testkit.Rows("1 2")) } -func (s *testSuite7) TestDefEnumInsert(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDefEnumInsert(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 test (id int, prescription_type enum('a','b','c','d','e','f') NOT NULL, primary key(id));") tk.MustExec("insert into test (id) values (1)") tk.MustQuery("select prescription_type from test").Check(testkit.Rows("a")) } -func (s *testSuite7) TestIssue11059(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue11059(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 (pk int primary key, uk int unique, v int)") tk.MustExec("insert into t values (2, 11, 215)") tk.MustExec("insert into t values (3, 7, 2111)") _, err := tk.Exec("update t set pk = 2 where uk = 7") - c.Assert(err, NotNil) + require.Error(t, err) } -func (s *testSuite7) TestSetWithRefGenCol(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestSetWithRefGenCol(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 (i int, j int as (i+1) not null);`) tk.MustExec(`insert into t set i = j + 1;`) @@ -3015,7 +2872,7 @@ func (s *testSuite7) TestSetWithRefGenCol(c *C) { tk.MustExec(`insert into t2 set i = j + 9`) tk.MustQuery("select * from t2").Check(testkit.Rows("10 9")) _, err := tk.Exec(`insert into t2 set j = i + 1`) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec(`insert into t2 set i = j + 100`) tk.MustQuery("select * from t2").Check(testkit.Rows("10 9", "101 100")) @@ -3023,11 +2880,14 @@ func (s *testSuite7) TestSetWithRefGenCol(c *C) { tk.MustExec(`insert into t3 set i = j + 100`) tk.MustQuery("select * from t3").Check(testkit.Rows(" ")) _, err = tk.Exec(`insert into t3 set j = i + 1`) - c.Assert(err, NotNil) + require.Error(t, err) } -func (s *testSuite7) TestSetWithCurrentTimestampAndNow(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestSetWithCurrentTimestampAndNow(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 tbl;`) tk.MustExec(`create table t1(c1 timestamp default current_timestamp, c2 int, c3 timestamp default current_timestamp);`) @@ -3038,8 +2898,11 @@ func (s *testSuite7) TestSetWithCurrentTimestampAndNow(c *C) { tk.MustQuery("select c1 = c3 from t1").Check(testkit.Rows("1", "1")) } -func (s *testSuite7) TestApplyWithPointAndBatchPointGet(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestApplyWithPointAndBatchPointGet(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 t;`) tk.MustExec(`create table t ( c_int int, c_str varchar(40),c_datetime datetime, c_timestamp timestamp, @@ -3079,8 +2942,11 @@ from t order by c_str;`).Check(testkit.Rows("10")) from t order by c_str;`).Check(testkit.Rows("10")) } -func (s *testSuite4) TestWriteListPartitionTable(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestWriteListPartitionTable(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_list_partition = ON") tk.MustExec("drop table if exists t") @@ -3110,9 +2976,9 @@ func (s *testSuite4) TestWriteListPartitionTable(c *C) { // Test insert error tk.MustExec("insert into t values (1, 'a')") _, err := tk.Exec("insert into t values (1, 'd')") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '1' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '1' for key 'idx'") _, err = tk.Exec("insert into t values (100, 'd')") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value 100") + require.EqualError(t, err, "[table:1526]Table has no partition for value 100") tk.MustExec("admin check table t;") // Test select partition @@ -3127,8 +2993,11 @@ func (s *testSuite4) TestWriteListPartitionTable(c *C) { tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 a", "2 b")) } -func (s *testSuite4) TestWriteListColumnsPartitionTable(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestWriteListColumnsPartitionTable(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_list_partition = ON") tk.MustExec("drop table if exists t") @@ -3158,9 +3027,9 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable(c *C) { // Test insert error tk.MustExec("insert into t values (1, 'a')") _, err := tk.Exec("insert into t values (1, 'd')") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '1' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '1' for key 'idx'") _, err = tk.Exec("insert into t values (100, 'd')") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value from column_list") + require.EqualError(t, err, "[table:1526]Table has no partition for value from column_list") tk.MustExec("admin check table t;") // Test select partition @@ -3176,8 +3045,11 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable(c *C) { } // TestWriteListPartitionTable1 test for write list partition when the partition expression is simple. -func (s *testSuite4) TestWriteListPartitionTable1(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestWriteListPartitionTable1(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_list_partition = ON") tk.MustExec("drop table if exists t") @@ -3191,7 +3063,7 @@ func (s *testSuite4) TestWriteListPartitionTable1(c *C) { // Test add unique index failed. tk.MustExec("insert into t values (1, 'a'),(1,'b')") _, err := tk.Exec("alter table t add unique index idx (id)") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '1' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '1' for key 'idx'") // Test add unique index success. tk.MustExec("delete from t where name='b'") tk.MustExec("alter table t add unique index idx (id)") @@ -3216,7 +3088,7 @@ func (s *testSuite4) TestWriteListPartitionTable1(c *C) { tk.MustQuery("select * from t partition(p3) order by id").Check(testkit.Rows()) // Test insert on duplicate error _, err = tk.Exec("insert into t values (3, 'a'), (11,'x') on duplicate key update id=id+1") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '4' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '4' for key 'idx'") tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 x", "3 x", "4 e", "5 g")) // Test insert ignore with duplicate tk.MustExec("insert ignore into t values (1, 'b'), (5,'a'),(null,'y')") @@ -3231,7 +3103,7 @@ func (s *testSuite4) TestWriteListPartitionTable1(c *C) { tk.MustQuery("select * from t partition(p3) order by id").Check(testkit.Rows(" y", "15 a")) // Test insert meet no partition error. _, err = tk.Exec("insert into t values (100, 'd')") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value 100") + require.EqualError(t, err, "[table:1526]Table has no partition for value 100") // --------------------------Test update--------------------------- // Test update 1 partition. @@ -3245,7 +3117,7 @@ func (s *testSuite4) TestWriteListPartitionTable1(c *C) { tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 y", "2 y", "3 c")) // Test update meet duplicate error. _, err = tk.Exec("update t set id=2 where id = 1") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '2' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '2' for key 'idx'") tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 y", "2 y", "3 c")) // Test update multi-partitions @@ -3257,11 +3129,11 @@ func (s *testSuite4) TestWriteListPartitionTable1(c *C) { tk.MustQuery("select * from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) // Test update meet duplicate error. _, err = tk.Exec("update t set id=id+17 where id in (3,10)") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '20' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '20' for key 'idx'") tk.MustQuery("select * from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) // Test update meet no partition error. _, err = tk.Exec("update t set id=id*2 where id in (3,20)") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value 40") + require.EqualError(t, err, "[table:1526]Table has no partition for value 40") tk.MustQuery("select * from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) // --------------------------Test replace--------------------------- @@ -3280,7 +3152,7 @@ func (s *testSuite4) TestWriteListPartitionTable1(c *C) { tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 x", "2 b", "3 c", "4 d", "7 x")) // Test replace meet no partition error. _, err = tk.Exec("replace into t values (10,'x'),(50,'x')") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value 50") + require.EqualError(t, err, "[table:1526]Table has no partition for value 50") tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 x", "2 b", "3 c", "4 d", "7 x")) // --------------------------Test delete--------------------------- @@ -3301,8 +3173,11 @@ func (s *testSuite4) TestWriteListPartitionTable1(c *C) { } // TestWriteListPartitionTable2 test for write list partition when the partition expression is complicated and contain generated column. -func (s *testSuite4) TestWriteListPartitionTable2(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestWriteListPartitionTable2(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_list_partition = ON") tk.MustExec("drop table if exists t") @@ -3317,7 +3192,7 @@ func (s *testSuite4) TestWriteListPartitionTable2(c *C) { // Test add unique index failed. tk.MustExec("insert into t (id,name) values (1, 'a'),(1,'b')") _, err := tk.Exec("alter table t add unique index idx (id,b)") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '1-2' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '1-2' for key 'idx'") // Test add unique index success. tk.MustExec("delete from t where name='b'") tk.MustExec("alter table t add unique index idx (id,b)") @@ -3342,7 +3217,7 @@ func (s *testSuite4) TestWriteListPartitionTable2(c *C) { tk.MustQuery("select id,name from t partition(p3) order by id").Check(testkit.Rows()) // Test insert on duplicate error _, err = tk.Exec("insert into t (id,name) values (3, 'a'), (11,'x') on duplicate key update id=id+1") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '4-2' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '4-2' for key 'idx'") tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 x", "3 x", "4 e", "5 g")) // Test insert ignore with duplicate tk.MustExec("insert ignore into t (id,name) values (1, 'b'), (5,'a'),(null,'y')") @@ -3357,7 +3232,7 @@ func (s *testSuite4) TestWriteListPartitionTable2(c *C) { tk.MustQuery("select id,name from t partition(p3) order by id").Check(testkit.Rows(" y", "15 a")) // Test insert meet no partition error. _, err = tk.Exec("insert into t (id,name) values (100, 'd')") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value 100") + require.EqualError(t, err, "[table:1526]Table has no partition for value 100") // --------------------------Test update--------------------------- // Test update 1 partition. @@ -3371,7 +3246,7 @@ func (s *testSuite4) TestWriteListPartitionTable2(c *C) { tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 y", "2 y", "3 c")) // Test update meet duplicate error. _, err = tk.Exec("update t set id=2 where id = 1") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '2-2' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '2-2' for key 'idx'") tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 y", "2 y", "3 c")) // Test update multi-partitions @@ -3383,11 +3258,11 @@ func (s *testSuite4) TestWriteListPartitionTable2(c *C) { tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) // Test update meet duplicate error. _, err = tk.Exec("update t set id=id+17 where id in (3,10)") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '20-2' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '20-2' for key 'idx'") tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) // Test update meet no partition error. _, err = tk.Exec("update t set id=id*2 where id in (3,20)") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value 40") + require.EqualError(t, err, "[table:1526]Table has no partition for value 40") tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) // --------------------------Test replace--------------------------- @@ -3406,7 +3281,7 @@ func (s *testSuite4) TestWriteListPartitionTable2(c *C) { tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 x", "2 b", "3 c", "4 d", "7 x")) // Test replace meet no partition error. _, err = tk.Exec("replace into t (id,name) values (10,'x'),(50,'x')") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value 50") + require.EqualError(t, err, "[table:1526]Table has no partition for value 50") tk.MustQuery("select id,name from t order by id").Check(testkit.Rows("1 x", "2 b", "3 c", "4 d", "7 x")) // --------------------------Test delete--------------------------- @@ -3426,8 +3301,11 @@ func (s *testSuite4) TestWriteListPartitionTable2(c *C) { tk.MustQuery("select id,name from t").Check(testkit.Rows()) } -func (s *testSuite4) TestWriteListColumnsPartitionTable1(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestWriteListColumnsPartitionTable1(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_list_partition = ON") @@ -3442,7 +3320,7 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable1(c *C) { // Test add unique index failed. tk.MustExec("insert into t values (1, 'a'),(1,'b')") _, err := tk.Exec("alter table t add unique index idx (id)") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '1' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '1' for key 'idx'") // Test add unique index success. tk.MustExec("delete from t where name='b'") tk.MustExec("alter table t add unique index idx (id)") @@ -3467,7 +3345,7 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable1(c *C) { tk.MustQuery("select * from t partition(p3) order by id").Check(testkit.Rows()) // Test insert on duplicate error _, err = tk.Exec("insert into t values (3, 'a'), (11,'x') on duplicate key update id=id+1") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '4' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '4' for key 'idx'") tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 x", "3 x", "4 e", "5 g")) // Test insert ignore with duplicate tk.MustExec("insert ignore into t values (1, 'b'), (5,'a'),(null,'y')") @@ -3482,7 +3360,7 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable1(c *C) { tk.MustQuery("select * from t partition(p3) order by id").Check(testkit.Rows(" y", "15 a")) // Test insert meet no partition error. _, err = tk.Exec("insert into t values (100, 'd')") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value from column_list") + require.EqualError(t, err, "[table:1526]Table has no partition for value from column_list") // --------------------------Test update--------------------------- // Test update 1 partition. @@ -3496,7 +3374,7 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable1(c *C) { tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 y", "2 y", "3 c")) // Test update meet duplicate error. _, err = tk.Exec("update t set id=2 where id = 1") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '2' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '2' for key 'idx'") tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 y", "2 y", "3 c")) // Test update multi-partitions @@ -3508,11 +3386,11 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable1(c *C) { tk.MustQuery("select * from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) // Test update meet duplicate error. _, err = tk.Exec("update t set id=id+17 where id in (3,10)") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '20' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '20' for key 'idx'") tk.MustQuery("select * from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) // Test update meet no partition error. _, err = tk.Exec("update t set id=id*2 where id in (3,20)") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value from column_list") + require.EqualError(t, err, "[table:1526]Table has no partition for value from column_list") tk.MustQuery("select * from t order by id").Check(testkit.Rows("3 a", "10 a", "20 a")) // --------------------------Test replace--------------------------- @@ -3531,7 +3409,7 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable1(c *C) { tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 x", "2 b", "3 c", "4 d", "7 x")) // Test replace meet no partition error. _, err = tk.Exec("replace into t values (10,'x'),(100,'x')") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value from column_list") + require.EqualError(t, err, "[table:1526]Table has no partition for value from column_list") tk.MustQuery("select * from t order by id").Check(testkit.Rows("1 x", "2 b", "3 c", "4 d", "7 x")) // --------------------------Test delete--------------------------- @@ -3552,8 +3430,11 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable1(c *C) { } // TestWriteListColumnsPartitionTable2 test for write list partition when the partition by multi-columns. -func (s *testSuite4) TestWriteListColumnsPartitionTable2(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestWriteListColumnsPartitionTable2(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_list_partition = ON") tk.MustExec("drop table if exists t") @@ -3567,7 +3448,7 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable2(c *C) { // Test add unique index failed. tk.MustExec("insert into t values ('w', 1, 1),('w', 1, 2)") _, err := tk.Exec("alter table t add unique index idx (location,id)") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry 'w-1' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry 'w-1' for key 'idx'") // Test add unique index success. tk.MustExec("delete from t where a=2") tk.MustExec("alter table t add unique index idx (location,id)") @@ -3594,7 +3475,7 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable2(c *C) { // Test insert on duplicate error tk.MustExec("insert into t values ('w', 2, 2), ('w', 1, 1)") _, err = tk.Exec("insert into t values ('w', 2, 3) on duplicate key update id=1") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry 'w-1' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry 'w-1' for key 'idx'") tk.MustQuery("select * from t partition(p_west) order by id").Check(testkit.Rows("w 1 1", "w 2 2")) // Test insert ignore with duplicate tk.MustExec("insert ignore into t values ('w', 2, 2), ('w', 3, 3), ('n', 10, 10)") @@ -3607,13 +3488,13 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable2(c *C) { tk.MustQuery("select * from t partition(p_south) order by id").Check(testkit.Rows("s 13 2", "s 14 14")) // Test insert meet no partition error. _, err = tk.Exec("insert into t values ('w', 5, 5)") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value from column_list") + require.EqualError(t, err, "[table:1526]Table has no partition for value from column_list") _, err = tk.Exec("insert into t values ('s', 5, 5)") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value from column_list") + require.EqualError(t, err, "[table:1526]Table has no partition for value from column_list") _, err = tk.Exec("insert into t values ('s', 100, 5)") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value from column_list") + require.EqualError(t, err, "[table:1526]Table has no partition for value from column_list") _, err = tk.Exec("insert into t values ('x', 1, 5)") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value from column_list") + require.EqualError(t, err, "[table:1526]Table has no partition for value from column_list") // --------------------------Test update--------------------------- // Test update 1 partition. @@ -3631,7 +3512,7 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable2(c *C) { tk.MustQuery("select * from t partition(p_west) order by id,a").Check(testkit.Rows("w 1 5", "w 2 5", "w 3 6")) // Test update meet duplicate error. _, err = tk.Exec("update t set id=id+1 where location='w' and id<2") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry 'w-2' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry 'w-2' for key 'idx'") tk.MustQuery("select * from t partition(p_west) order by id,a").Check(testkit.Rows("w 1 5", "w 2 5", "w 3 6")) // Test update multi-partitions @@ -3649,11 +3530,11 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable2(c *C) { tk.MustQuery("select * from t order by id").Check(testkit.Rows("w 1 4", "w 2 4", "e 8 9", "n 11 15")) // Test update meet duplicate error. _, err = tk.Exec("update t set id=id+1 where location='w' and id in (1,2)") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry 'w-2' for key 'idx'") + require.EqualError(t, err, "[kv:1062]Duplicate entry 'w-2' for key 'idx'") tk.MustQuery("select * from t order by id").Check(testkit.Rows("w 1 4", "w 2 4", "e 8 9", "n 11 15")) // Test update meet no partition error. _, err = tk.Exec("update t set id=id+3 where location='w' and id in (1,2)") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value from column_list") + require.EqualError(t, err, "[table:1526]Table has no partition for value from column_list") tk.MustQuery("select * from t order by id").Check(testkit.Rows("w 1 4", "w 2 4", "e 8 9", "n 11 15")) // Test update that move from partition 1 to partition 2. // TODO: fix this @@ -3674,13 +3555,13 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable2(c *C) { tk.MustQuery("select * from t order by id").Check(testkit.Rows("w 1 2", "e 5 5", "n 9 9", "n 10 10")) // Test replace meet no partition error. _, err = tk.Exec("replace into t values ('w', 5, 5)") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value from column_list") + require.EqualError(t, err, "[table:1526]Table has no partition for value from column_list") _, err = tk.Exec("replace into t values ('s', 5, 5)") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value from column_list") + require.EqualError(t, err, "[table:1526]Table has no partition for value from column_list") _, err = tk.Exec("replace into t values ('s', 100, 5)") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value from column_list") + require.EqualError(t, err, "[table:1526]Table has no partition for value from column_list") _, err = tk.Exec("replace into t values ('x', 1, 5)") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value from column_list") + require.EqualError(t, err, "[table:1526]Table has no partition for value from column_list") // --------------------------Test delete--------------------------- // Test delete 1 partition. @@ -3701,18 +3582,24 @@ func (s *testSuite4) TestWriteListColumnsPartitionTable2(c *C) { } // TestWriteListColumnsPartitionTable2 test for write list partition when the partition by multi-columns. -func (s *testSuite4) TestWriteListPartitionTableIssue21437(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestWriteListPartitionTableIssue21437(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_list_partition = ON") tk.MustExec("drop table if exists t") tk.MustExec(`create table t (a int) partition by list (a%10) (partition p0 values in (0,1));`) _, err := tk.Exec("replace into t values (null)") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value NULL") + require.EqualError(t, err, "[table:1526]Table has no partition for value NULL") } -func (s *testSuite4) TestListPartitionWithAutoRandom(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestListPartitionWithAutoRandom(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_list_partition = ON") tk.MustExec("drop table if exists t") @@ -3734,8 +3621,11 @@ func (s *testSuite4) TestListPartitionWithAutoRandom(c *C) { tk.MustQuery("select b from t where a=1").Check(testkit.Rows("4")) } -func (s *testSuite4) TestListPartitionWithAutoIncrement(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestListPartitionWithAutoIncrement(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_list_partition = ON") tk.MustExec("drop table if exists t") @@ -3757,8 +3647,11 @@ func (s *testSuite4) TestListPartitionWithAutoIncrement(c *C) { tk.MustQuery("select b from t where a=1").Check(testkit.Rows("4")) } -func (s *testSuite4) TestListPartitionWithGeneratedColumn(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestListPartitionWithGeneratedColumn(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_list_partition = ON") // Test for generated column with bigint type. @@ -3795,15 +3688,18 @@ func (s *testSuite4) TestListPartitionWithGeneratedColumn(c *C) { // Test for insert meet no partition error _, err := tk.Exec("insert into t (a) values (11)") - c.Assert(table.ErrNoPartitionForGivenValue.Equal(err), IsTrue) + require.True(t, table.ErrNoPartitionForGivenValue.Equal(err)) // Test for update meet no partition error _, err = tk.Exec("update t set a=a+10 where a = 2") - c.Assert(table.ErrNoPartitionForGivenValue.Equal(err), IsTrue) + require.True(t, table.ErrNoPartitionForGivenValue.Equal(err)) } } -func (s *testSuite4) TestListPartitionWithGeneratedColumn1(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestListPartitionWithGeneratedColumn1(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_list_partition = ON") // Test for generated column with year type. @@ -3836,11 +3732,10 @@ func (s *testSuite4) TestListPartitionWithGeneratedColumn1(c *C) { // Test for insert meet no partition error _, err := tk.Exec("insert into t (a) values (11)") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value 2011") + require.EqualError(t, err, "[table:1526]Table has no partition for value 2011") // Test for update meet no partition error _, err = tk.Exec("update t set a=a+10 where a = 2") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value 2012") - + require.EqualError(t, err, "[table:1526]Table has no partition for value 2012") tk.MustExec("delete from t") // Test for insert @@ -3863,15 +3758,18 @@ func (s *testSuite4) TestListPartitionWithGeneratedColumn1(c *C) { // Test for insert meet no partition error _, err = tk.Exec("insert into t (a) values (2011)") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value 2011") + require.EqualError(t, err, "[table:1526]Table has no partition for value 2011") // Test for update meet no partition error _, err = tk.Exec("update t set a=a+10 where a = 2002") - c.Assert(err.Error(), Equals, "[table:1526]Table has no partition for value 2012") + require.EqualError(t, err, "[table:1526]Table has no partition for value 2012") } } -func (s *testSuite4) TestListPartitionWithGeneratedColumn2(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestListPartitionWithGeneratedColumn2(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_list_partition = ON") tableDefs := []string{ @@ -3912,8 +3810,11 @@ func (s *testSuite4) TestListPartitionWithGeneratedColumn2(c *C) { } } -func (s *testSuite4) TestListColumnsPartitionWithGeneratedColumn(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestListColumnsPartitionWithGeneratedColumn(t *testing.T) { + t.Parallel() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_list_partition = ON") // Test for generated column with substr expression. @@ -3936,111 +3837,23 @@ func (s *testSuite4) TestListColumnsPartitionWithGeneratedColumn(c *C) { } } -func (s *testSerialSuite2) TestListColumnsPartitionWithGlobalIndex(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("use test") - tk.MustExec("set @@session.tidb_enable_list_partition = ON") - // Test generated column with global index - restoreConfig := config.RestoreFunc() - defer restoreConfig() - config.UpdateGlobal(func(conf *config.Config) { - conf.EnableGlobalIndex = true - }) - tableDefs := []string{ - // Test for virtual generated column with global index - `create table t (a varchar(10), b varchar(1) GENERATED ALWAYS AS (substr(a,1,1)) VIRTUAL) partition by list columns(b) (partition p0 values in ('a','c'), partition p1 values in ('b','d'));`, - // Test for stored generated column with global index - `create table t (a varchar(10), b varchar(1) GENERATED ALWAYS AS (substr(a,1,1)) STORED) partition by list columns(b) (partition p0 values in ('a','c'), partition p1 values in ('b','d'));`, - } - for _, tbl := range tableDefs { - tk.MustExec("drop table if exists t") - tk.MustExec(tbl) - tk.MustExec("alter table t add unique index (a)") - tk.MustExec("insert into t (a) values ('aaa'),('abc'),('acd')") - tk.MustQuery("select a from t partition (p0) order by a").Check(testkit.Rows("aaa", "abc", "acd")) - tk.MustQuery("select * from t where a = 'abc' order by a").Check(testkit.Rows("abc a")) - tk.MustExec("update t set a='bbb' where a = 'aaa'") - tk.MustExec("admin check table t") - tk.MustQuery("select a from t order by a").Check(testkit.Rows("abc", "acd", "bbb")) - // TODO: fix below test. - //tk.MustQuery("select a from t partition (p0) order by a").Check(testkit.Rows("abc", "acd")) - //tk.MustQuery("select a from t partition (p1) order by a").Check(testkit.Rows("bbb")) - tk.MustQuery("select * from t where a = 'bbb' order by a").Check(testkit.Rows("bbb b")) - // Test insert meet duplicate error. - _, err := tk.Exec("insert into t (a) values ('abc')") - c.Assert(err, NotNil) - // Test insert on duplicate update - tk.MustExec("insert into t (a) values ('abc') on duplicate key update a='bbc'") - tk.MustQuery("select a from t order by a").Check(testkit.Rows("acd", "bbb", "bbc")) - tk.MustQuery("select * from t where a = 'bbc'").Check(testkit.Rows("bbc b")) - // TODO: fix below test. - //tk.MustQuery("select a from t partition (p0) order by a").Check(testkit.Rows("acd")) - //tk.MustQuery("select a from t partition (p1) order by a").Check(testkit.Rows("bbb", "bbc")) - } -} - -func (s *testSerialSuite) TestIssue20724(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1") - tk.MustExec("create table t1(a varchar(10) collate utf8mb4_general_ci)") - tk.MustExec("insert into t1 values ('a')") - tk.MustExec("update t1 set a = 'A'") - tk.MustQuery("select * from t1").Check(testkit.Rows("A")) - tk.MustExec("drop table t1") -} - -func (s *testSerialSuite) TestIssue20840(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly - tk.MustExec("create table t1 (i varchar(20) unique key) collate=utf8mb4_general_ci") - tk.MustExec("insert into t1 values ('a')") - tk.MustExec("replace into t1 values ('A')") - tk.MustQuery("select * from t1").Check(testkit.Rows("A")) - tk.MustExec("drop table t1") -} - -func (s *testSerialSuite) TestIssueInsertPrefixIndexForNonUTF8Collation(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2, t3") - tk.MustExec("create table t1 ( c_int int, c_str varchar(40) character set ascii collate ascii_bin, primary key(c_int, c_str(8)) clustered , unique key(c_str))") - tk.MustExec("create table t2 ( c_int int, c_str varchar(40) character set latin1 collate latin1_bin, primary key(c_int, c_str(8)) clustered , unique key(c_str))") - tk.MustExec("insert into t1 values (3, 'fervent brattain')") - tk.MustExec("insert into t2 values (3, 'fervent brattain')") - tk.MustExec("admin check table t1") - tk.MustExec("admin check table t2") - - tk.MustExec("create table t3 (x varchar(40) CHARACTER SET ascii COLLATE ascii_bin, UNIQUE KEY uk(x(4)))") - tk.MustExec("insert into t3 select 'abc '") - tk.MustGetErrCode("insert into t3 select 'abc d'", 1062) -} - -func (s *testSerialSuite) TestIssue22496(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue22496(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 t12") tk.MustExec("create table t12(d decimal(15,2));") _, err := tk.Exec("insert into t12 values('1,9999.00')") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("set sql_mode=''") tk.MustExec("insert into t12 values('1,999.00');") tk.MustQuery("SELECT * FROM t12;").Check(testkit.Rows("1.00")) tk.MustExec("drop table t12") } -func (s *testSuite) TestEqualDatumsAsBinary(c *C) { +func TestEqualDatumsAsBinary(t *testing.T) { tests := []struct { a []interface{} b []interface{} @@ -4064,12 +3877,15 @@ func (s *testSuite) TestEqualDatumsAsBinary(c *C) { {[]interface{}{nil}, []interface{}{1}, false}, } for _, tt := range tests { - testEqualDatumsAsBinary(c, tt.a, tt.b, tt.same) + testEqualDatumsAsBinary(t, tt.a, tt.b, tt.same) } } -func (s *testSuite) TestIssue21232(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue21232(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 t, t1") tk.MustExec("create table t(a varchar(1), index idx(a))") @@ -4087,11 +3903,11 @@ func (s *testSuite) TestIssue21232(c *C) { tk.MustQuery("select * from t").Check(testkit.Rows("a", "b")) } -func testEqualDatumsAsBinary(c *C, a []interface{}, b []interface{}, same bool) { +func testEqualDatumsAsBinary(t *testing.T, a []interface{}, b []interface{}, same bool) { sc := new(stmtctx.StatementContext) re := new(executor.ReplaceExec) sc.IgnoreTruncate = true res, err := re.EqualDatumsAsBinary(sc, types.MakeDatums(a...), types.MakeDatums(b...)) - c.Assert(err, IsNil) - c.Assert(res, Equals, same, Commentf("a: %v, b: %v", a, b)) + require.NoError(t, err) + require.Equal(t, same, res, "a: %v, b: %v", a, b) } diff --git a/expression/builtin.go b/expression/builtin.go index a7e9537e84a52..316e3a2ecb462 100644 --- a/expression/builtin.go +++ b/expression/builtin.go @@ -140,7 +140,7 @@ func newBaseBuiltinFuncWithTp(ctx sessionctx.Context, funcName string, args []Ex args[i] = WrapWithCastAsDecimal(ctx, args[i]) case types.ETString: args[i] = WrapWithCastAsString(ctx, args[i]) - args[i] = WrapWithToBinary(ctx, args[i], funcName) + args[i] = HandleBinaryLiteral(ctx, args[i], ec, funcName) case types.ETDatetime: args[i] = WrapWithCastAsTime(ctx, args[i], types.NewFieldType(mysql.TypeDatetime)) case types.ETTimestamp: @@ -880,9 +880,6 @@ var funcs = map[string]functionClass{ ast.NextVal: &nextValFunctionClass{baseFunctionClass{ast.NextVal, 1, 1}}, ast.LastVal: &lastValFunctionClass{baseFunctionClass{ast.LastVal, 1, 1}}, ast.SetVal: &setValFunctionClass{baseFunctionClass{ast.SetVal, 2, 2}}, - - // TiDB implicit internal functions. - InternalFuncToBinary: &tidbConvertCharsetFunctionClass{baseFunctionClass{InternalFuncToBinary, 1, 1}}, } // IsFunctionSupported check if given function name is a builtin sql function. @@ -906,7 +903,6 @@ func GetDisplayName(name string) string { func GetBuiltinList() []string { res := make([]string, 0, len(funcs)) notImplementedFunctions := []string{ast.RowFunc, ast.IsTruthWithNull} - implicitFunctions := []string{InternalFuncToBinary} for funcName := range funcs { skipFunc := false // Skip not implemented functions @@ -915,11 +911,6 @@ func GetBuiltinList() []string { skipFunc = true } } - for _, implicitFunc := range implicitFunctions { - if funcName == implicitFunc { - skipFunc = true - } - } // Skip literal functions // (their names are not readable: 'tidb`.(dateliteral, for example) // See: https://github.com/pingcap/parser/pull/591 diff --git a/expression/builtin_cast.go b/expression/builtin_cast.go index 407982af11e31..0ec4ad904dee4 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -292,6 +292,9 @@ func (c *castAsStringFunctionClass) getFunction(ctx sessionctx.Context, args []E argTp := args[0].GetType().EvalType() switch argTp { case types.ETInt: + if bf.tp.Flen == types.UnspecifiedLength { + bf.tp.Flen = args[0].GetType().Flen + } sig = &builtinCastIntAsStringSig{bf} sig.setPbCode(tipb.ScalarFuncSig_CastIntAsString) case types.ETReal: @@ -1915,17 +1918,17 @@ func WrapWithCastAsString(ctx sessionctx.Context, expr Expression) Expression { return expr } argLen := exprTp.Flen - // If expr is decimal, we should take the decimal point and negative sign - // into consideration, so we set `expr.GetType().Flen + 2` as the `argLen`. + // If expr is decimal, we should take the decimal point ,negative sign and the leading zero(0.xxx) + // into consideration, so we set `expr.GetType().Flen + 3` as the `argLen`. // Since the length of float and double is not accurate, we do not handle // them. if exprTp.Tp == mysql.TypeNewDecimal && argLen != int(types.UnspecifiedFsp) { - argLen += 2 + argLen += 3 } if exprTp.EvalType() == types.ETInt { argLen = mysql.MaxIntWidth } - // because we can't control the length of cast(float as char) for now, we can't determine the argLen + // Because we can't control the length of cast(float as char) for now, we can't determine the argLen. if exprTp.Tp == mysql.TypeFloat || exprTp.Tp == mysql.TypeDouble { argLen = -1 } diff --git a/expression/builtin_compare.go b/expression/builtin_compare.go index dce8d2cdd3ba0..1090ca0e14eea 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tipb/go-tipb" ) @@ -1378,7 +1379,7 @@ func RefineComparedConstant(ctx sessionctx.Context, targetFieldType types.FieldT } return con, false } - c, err := intDatum.CompareDatum(sc, &con.Value) + c, err := intDatum.Compare(sc, &con.Value, collate.GetBinaryCollator()) if err != nil { return con, false } diff --git a/expression/builtin_convert_charset.go b/expression/builtin_convert_charset.go index 243a50e5d0a4e..aac9d1680ddda 100644 --- a/expression/builtin_convert_charset.go +++ b/expression/builtin_convert_charset.go @@ -16,24 +16,43 @@ package expression import ( "fmt" + "unicode/utf8" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tipb/go-tipb" ) +var ( + _ functionClass = &tidbToBinaryFunctionClass{} + _ functionClass = &tidbFromBinaryFunctionClass{} + + _ builtinFunc = &builtinInternalToBinarySig{} + _ builtinFunc = &builtinInternalFromBinarySig{} +) + +var ( + // errCannotConvertString returns when the string can not convert to other charset. + errCannotConvertString = dbterror.ClassExpression.NewStd(errno.ErrCannotConvertString) +) + // InternalFuncToBinary accepts a string and returns another string encoded in a given charset. const InternalFuncToBinary = "to_binary" -type tidbConvertCharsetFunctionClass struct { +// InternalFuncFromBinary accepts a string and returns another string decode in a given charset. +const InternalFuncFromBinary = "from_binary" + +type tidbToBinaryFunctionClass struct { baseFunctionClass } -func (c *tidbConvertCharsetFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { +func (c *tidbToBinaryFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { if err := c.verifyArgs(args); err != nil { return nil, c.verifyArgs(args) } @@ -45,6 +64,8 @@ func (c *tidbConvertCharsetFunctionClass) getFunction(ctx sessionctx.Context, ar if err != nil { return nil, err } + bf.tp = args[0].GetType().Clone() + bf.tp.Charset, bf.tp.Collate = charset.CharsetBin, charset.CollationBin sig = &builtinInternalToBinarySig{bf} sig.setPbCode(tipb.ScalarFuncSig_ToBinary) default: @@ -53,8 +74,6 @@ func (c *tidbConvertCharsetFunctionClass) getFunction(ctx sessionctx.Context, ar return sig, nil } -var _ builtinFunc = &builtinInternalToBinarySig{} - type builtinInternalToBinarySig struct { baseBuiltinFunc } @@ -92,44 +111,170 @@ func (b *builtinInternalToBinarySig) vecEvalString(input *chunk.Chunk, result *c } enc := charset.NewEncoding(b.args[0].GetType().Charset) result.ReserveString(n) + var encodedBuf []byte for i := 0; i < n; i++ { - var str string if buf.IsNull(i) { result.AppendNull() continue } - str = buf.GetString(i) - str, err = enc.EncodeString(str) + strBytes, err := enc.Encode(encodedBuf, buf.GetBytes(i)) if err != nil { return err } - result.AppendString(str) + result.AppendBytes(strBytes) } return nil } -// toBinaryMap contains the builtin functions which arguments need to be converted to the correct charset. -var toBinaryMap = map[string]struct{}{ - ast.Hex: {}, ast.Length: {}, ast.OctetLength: {}, ast.ASCII: {}, - ast.ToBase64: {}, +type tidbFromBinaryFunctionClass struct { + baseFunctionClass + + tp *types.FieldType } -// WrapWithToBinary wraps `expr` with to_binary sig. -func WrapWithToBinary(ctx sessionctx.Context, expr Expression, funcName string) Expression { - exprTp := expr.GetType() - if _, err := charset.GetDefaultCollationLegacy(exprTp.Charset); err != nil { - if _, ok := toBinaryMap[funcName]; ok { - fc := funcs[InternalFuncToBinary] - sig, err := fc.getFunction(ctx, []Expression{expr}) - if err != nil { - return expr +func (c *tidbFromBinaryFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { + if err := c.verifyArgs(args); err != nil { + return nil, c.verifyArgs(args) + } + argTp := args[0].GetType().EvalType() + var sig builtinFunc + switch argTp { + case types.ETString: + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETString) + if err != nil { + return nil, err + } + bf.tp = c.tp + sig = &builtinInternalFromBinarySig{bf} + sig.setPbCode(tipb.ScalarFuncSig_FromBinary) + default: + return nil, fmt.Errorf("unexpected argTp: %d", argTp) + } + return sig, nil +} + +type builtinInternalFromBinarySig struct { + baseBuiltinFunc +} + +func (b *builtinInternalFromBinarySig) Clone() builtinFunc { + newSig := &builtinInternalFromBinarySig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinInternalFromBinarySig) evalString(row chunk.Row) (res string, isNull bool, err error) { + val, isNull, err := b.args[0].EvalString(b.ctx, row) + if isNull || err != nil { + return val, isNull, err + } + transferString := b.getTransferFunc() + tBytes, err := transferString([]byte(val)) + return string(tBytes), false, err +} + +func (b *builtinInternalFromBinarySig) vectorized() bool { + return true +} + +func (b *builtinInternalFromBinarySig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + buf, err := b.bufAllocator.get() + if err != nil { + return err + } + defer b.bufAllocator.put(buf) + if err := b.args[0].VecEvalString(b.ctx, input, buf); err != nil { + return err + } + transferString := b.getTransferFunc() + result.ReserveString(n) + for i := 0; i < n; i++ { + if buf.IsNull(i) { + result.AppendNull() + continue + } + str, err := transferString(buf.GetBytes(i)) + if err != nil { + return err + } + result.AppendBytes(str) + } + return nil +} + +func (b *builtinInternalFromBinarySig) getTransferFunc() func([]byte) ([]byte, error) { + var transferString func([]byte) ([]byte, error) + if b.tp.Charset == charset.CharsetUTF8MB4 || b.tp.Charset == charset.CharsetUTF8 { + transferString = func(s []byte) ([]byte, error) { + if !utf8.Valid(s) { + return nil, errCannotConvertString.GenWithStackByArgs(fmt.Sprintf("%X", s), charset.CharsetBin, b.tp.Charset) } - sf := &ScalarFunction{ - FuncName: model.NewCIStr(InternalFuncToBinary), - RetType: exprTp, - Function: sig, + return s, nil + } + } else { + enc := charset.NewEncoding(b.tp.Charset) + var buf []byte + transferString = func(s []byte) ([]byte, error) { + str, err := enc.Decode(buf, s) + if err != nil { + return nil, errCannotConvertString.GenWithStackByArgs(fmt.Sprintf("%X", s), charset.CharsetBin, b.tp.Charset) } - return FoldConstant(sf) + return str, nil + } + } + return transferString +} + +// BuildToBinaryFunction builds to_binary function. +func BuildToBinaryFunction(ctx sessionctx.Context, expr Expression) (res Expression) { + fc := &tidbToBinaryFunctionClass{baseFunctionClass{InternalFuncToBinary, 1, 1}} + f, err := fc.getFunction(ctx, []Expression{expr}) + if err != nil { + return expr + } + res = &ScalarFunction{ + FuncName: model.NewCIStr(InternalFuncToBinary), + RetType: f.getRetTp(), + Function: f, + } + return FoldConstant(res) +} + +// BuildFromBinaryFunction builds from_binary function. +func BuildFromBinaryFunction(ctx sessionctx.Context, expr Expression, tp *types.FieldType) (res Expression) { + fc := &tidbFromBinaryFunctionClass{baseFunctionClass{InternalFuncFromBinary, 1, 1}, tp} + f, err := fc.getFunction(ctx, []Expression{expr}) + if err != nil { + return expr + } + res = &ScalarFunction{ + FuncName: model.NewCIStr(InternalFuncFromBinary), + RetType: tp, + Function: f, + } + return FoldConstant(res) +} + +// HandleBinaryLiteral wraps `expr` with to_binary or from_binary sig. +func HandleBinaryLiteral(ctx sessionctx.Context, expr Expression, ec *ExprCollation, funcName string) Expression { + switch funcName { + case ast.Concat, ast.ConcatWS, ast.Lower, ast.Lcase, ast.Reverse, ast.Upper, ast.Ucase, ast.Quote, ast.Coalesce, + ast.Left, ast.Right, ast.Repeat, ast.Trim, ast.LTrim, ast.RTrim, ast.Substr, ast.SubstringIndex, ast.Replace, + ast.Substring, ast.Mid, ast.Translate, ast.InsertFunc, ast.Lpad, ast.Rpad, ast.Elt, ast.ExportSet, ast.MakeSet, + ast.FindInSet, ast.Regexp, ast.Field, ast.Locate, ast.Instr, ast.Position, ast.GE, ast.LE, ast.GT, ast.LT, ast.EQ, + ast.NE, ast.NullEQ, ast.Strcmp, ast.If, ast.Ifnull, ast.Like, ast.In, ast.DateFormat, ast.TimeFormat: + if ec.Charset == charset.CharsetBin && expr.GetType().Charset != charset.CharsetBin { + return BuildToBinaryFunction(ctx, expr) + } else if ec.Charset != charset.CharsetBin && expr.GetType().Charset == charset.CharsetBin { + ft := expr.GetType().Clone() + ft.Charset, ft.Collate = ec.Charset, ec.Collation + return BuildFromBinaryFunction(ctx, expr, ft) + } + case ast.Hex, ast.Length, ast.OctetLength, ast.ASCII, ast.ToBase64, ast.AesDecrypt, ast.Decode, ast.Encode, + ast.PasswordFunc, ast.MD5, ast.SHA, ast.SHA1, ast.SHA2, ast.Compress: + if _, err := charset.GetDefaultCollationLegacy(expr.GetType().Charset); err != nil { + return BuildToBinaryFunction(ctx, expr) } } return expr diff --git a/expression/builtin_encryption.go b/expression/builtin_encryption.go index 75dfa7d179561..134b2c417591c 100644 --- a/expression/builtin_encryption.go +++ b/expression/builtin_encryption.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/parser/auth" - "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" @@ -159,18 +158,10 @@ 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")) @@ -210,28 +201,16 @@ 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) } @@ -427,21 +406,11 @@ func (b *builtinDecodeSig) evalString(row chunk.Row) (string, bool, error) { if isNull || err != nil { return "", true, err } - dataTp := b.args[0].GetType() - dataStr, err = charset.NewEncoding(dataTp.Charset).EncodeString(dataStr) - if err != nil { - return "", false, err - } passwordStr, isNull, err := b.args[1].EvalString(b.ctx, row) if isNull || err != nil { return "", true, err } - passwordTp := b.args[1].GetType() - passwordStr, err = charset.NewEncoding(passwordTp.Charset).EncodeString(passwordStr) - if err != nil { - return "", false, err - } decodeStr, err := encrypt.SQLDecode(dataStr, passwordStr) return decodeStr, false, err @@ -500,21 +469,11 @@ func (b *builtinEncodeSig) evalString(row chunk.Row) (string, bool, error) { if isNull || err != nil { return "", true, err } - decodeTp := b.args[0].GetType() - decodeStr, err = charset.NewEncoding(decodeTp.Charset).EncodeString(decodeStr) - if err != nil { - return "", false, err - } passwordStr, isNull, err := b.args[1].EvalString(b.ctx, row) if isNull || err != nil { return "", true, err } - passwordTp := b.args[1].GetType() - passwordStr, err = charset.NewEncoding(passwordTp.Charset).EncodeString(passwordStr) - if err != nil { - return "", false, err - } dataStr, err := encrypt.SQLEncode(decodeStr, passwordStr) return dataStr, false, err @@ -576,16 +535,11 @@ func (b *builtinPasswordSig) evalString(row chunk.Row) (d string, isNull bool, e return "", false, nil } - dStr, err := charset.NewEncoding(b.args[0].GetType().Charset).EncodeString(pass) - if err != nil { - return "", false, err - } - // We should append a warning here because function "PASSWORD" is deprecated since MySQL 5.7.6. // See https://dev.mysql.com/doc/refman/5.7/en/encryption-functions.html#function_password b.ctx.GetSessionVars().StmtCtx.AppendWarning(errDeprecatedSyntaxNoReplacement.GenWithStackByArgs("PASSWORD")) - return auth.EncodePassword(dStr), false, nil + return auth.EncodePassword(pass), false, nil } type randomBytesFunctionClass struct { @@ -671,12 +625,7 @@ func (b *builtinMD5Sig) evalString(row chunk.Row) (string, bool, error) { if isNull || err != nil { return "", isNull, err } - var sum [16]byte - dBytes, err := charset.NewEncoding(b.args[0].GetType().Charset).Encode(nil, []byte(arg)) - if err != nil { - return "", false, err - } - sum = md5.Sum(dBytes) // #nosec G401 + sum := md5.Sum([]byte(arg)) // #nosec G401 hexStr := fmt.Sprintf("%x", sum) return hexStr, false, nil } @@ -718,12 +667,8 @@ func (b *builtinSHA1Sig) evalString(row chunk.Row) (string, bool, error) { if isNull || err != nil { return "", isNull, err } - bytes, err := charset.NewEncoding(b.args[0].GetType().Charset).Encode(nil, []byte(str)) - if err != nil { - return "", false, err - } hasher := sha1.New() // #nosec G401 - _, err = hasher.Write(bytes) + _, err = hasher.Write([]byte(str)) if err != nil { return "", true, err } @@ -775,10 +720,6 @@ func (b *builtinSHA2Sig) evalString(row chunk.Row) (string, bool, error) { if isNull || err != nil { return "", isNull, err } - bytes, err := charset.NewEncoding(b.args[0].GetType().Charset).Encode(nil, []byte(str)) - if err != nil { - return "", false, err - } hashLength, isNull, err := b.args[1].EvalInt(b.ctx, row) if isNull || err != nil { return "", isNull, err @@ -799,7 +740,7 @@ func (b *builtinSHA2Sig) evalString(row chunk.Row) (string, bool, error) { return "", true, nil } - _, err = hasher.Write(bytes) + _, err = hasher.Write([]byte(str)) if err != nil { return "", true, err } @@ -876,11 +817,6 @@ 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_vec.go b/expression/builtin_encryption_vec.go index 9d076a3234232..4433463429127 100644 --- a/expression/builtin_encryption_vec.go +++ b/expression/builtin_encryption_vec.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/parser/auth" - "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/encrypt" @@ -67,19 +66,9 @@ 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) - cryptEnc := charset.NewEncoding(b.args[0].GetType().Charset) - keyEnc := charset.NewEncoding(b.args[1].GetType().Charset) - - var ( - key []byte - encodedBuf []byte - ) + var key []byte if isConstKey { - keyBytes, err := keyEnc.Encode(encodedBuf, keyBuf.GetBytes(0)) - if err != nil { - return err - } - key = encrypt.DeriveKeyMySQL(keyBytes, b.keySize) + key = encrypt.DeriveKeyMySQL(keyBuf.GetBytes(0), b.keySize) } result.ReserveString(n) @@ -95,19 +84,12 @@ func (b *builtinAesDecryptSig) vecEvalString(input *chunk.Chunk, result *chunk.C stmtCtx.AppendWarning(errWarnOptionIgnored.GenWithStackByArgs("IV")) } if !isConstKey { - keyBytes, err := keyEnc.Encode(encodedBuf, keyBuf.GetBytes(i)) - if err != nil { - return err - } - key = encrypt.DeriveKeyMySQL(keyBytes, b.keySize) + key = encrypt.DeriveKeyMySQL(keyBuf.GetBytes(i), 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, err := cryptEnc.Encode(encodedBuf, []byte(strBuf.GetString(i))) - if err != nil { - return err - } + str := []byte(strBuf.GetString(i)) plainText, err := encrypt.AESDecryptWithECB(str, key) if err != nil { result.AppendNull() @@ -228,9 +210,6 @@ func (b *builtinDecodeSig) vecEvalString(input *chunk.Chunk, result *chunk.Colum if err := b.args[0].VecEvalString(b.ctx, input, buf); err != nil { return err } - dataTp := b.args[0].GetType() - dataEnc := charset.NewEncoding(dataTp.Charset) - buf1, err1 := b.bufAllocator.get() if err1 != nil { return err1 @@ -239,22 +218,14 @@ func (b *builtinDecodeSig) vecEvalString(input *chunk.Chunk, result *chunk.Colum if err := b.args[1].VecEvalString(b.ctx, input, buf1); err != nil { return err } - passwordTp := b.args[1].GetType() - passwordEnc := charset.NewEncoding(passwordTp.Charset) result.ReserveString(n) for i := 0; i < n; i++ { if buf.IsNull(i) || buf1.IsNull(i) { result.AppendNull() continue } - dataStr, err := dataEnc.EncodeString(buf.GetString(i)) - if err != nil { - return err - } - passwordStr, err := passwordEnc.EncodeString(buf1.GetString(i)) - if err != nil { - return err - } + dataStr := buf.GetString(i) + passwordStr := buf1.GetString(i) decodeStr, err := encrypt.SQLDecode(dataStr, passwordStr) if err != nil { return err @@ -282,29 +253,18 @@ func (b *builtinEncodeSig) vecEvalString(input *chunk.Chunk, result *chunk.Colum if err1 != nil { return err1 } - dataTp := b.args[0].GetType() - dataEnc := charset.NewEncoding(dataTp.Charset) defer b.bufAllocator.put(buf1) if err := b.args[1].VecEvalString(b.ctx, input, buf1); err != nil { return err } - passwordTp := b.args[1].GetType() - passwordEnc := charset.NewEncoding(passwordTp.Charset) result.ReserveString(n) for i := 0; i < n; i++ { if buf.IsNull(i) || buf1.IsNull(i) { result.AppendNull() continue } - - decodeStr, err := dataEnc.EncodeString(buf.GetString(i)) - if err != nil { - return err - } - passwordStr, err := passwordEnc.EncodeString(buf1.GetString(i)) - if err != nil { - return err - } + decodeStr := buf.GetString(i) + passwordStr := buf1.GetString(i) dataStr, err := encrypt.SQLEncode(decodeStr, passwordStr) if err != nil { return err @@ -349,10 +309,6 @@ 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 @@ -368,19 +324,9 @@ func (b *builtinAesDecryptIVSig) vecEvalString(input *chunk.Chunk, result *chunk } isConst := b.args[1].ConstItem(b.ctx.GetSessionVars().StmtCtx) - 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 - ) + var key []byte if isConst { - keyBytes, err := keyEnc.Encode(encodedBuf, keyBuf.GetBytes(0)) - if err != nil { - return err - } - key = encrypt.DeriveKeyMySQL(keyBytes, b.keySize) + key = encrypt.DeriveKeyMySQL(keyBuf.GetBytes(0), b.keySize) } result.ReserveString(n) @@ -391,39 +337,28 @@ func (b *builtinAesDecryptIVSig) vecEvalString(input *chunk.Chunk, result *chunk continue } - iv, err := ivEnc.Encode(ivEncodedBuf, ivBuf.GetBytes(i)) - if err != nil { - return err - } + iv := ivBuf.GetBytes(i) 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 { - keyBytes, err := keyEnc.Encode(encodedBuf, keyBuf.GetBytes(i)) - if err != nil { - return err - } - key = encrypt.DeriveKeyMySQL(keyBytes, b.keySize) + key = encrypt.DeriveKeyMySQL(keyBuf.GetBytes(i), 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(str, key, iv) + plainText, err = encrypt.AESDecryptWithCBC([]byte(strBuf.GetString(i)), key, iv) } if isOFB { - plainText, err = encrypt.AESDecryptWithOFB(str, key, iv) + plainText, err = encrypt.AESDecryptWithOFB([]byte(strBuf.GetString(i)), key, iv) } if isCFB { - plainText, err = encrypt.AESDecryptWithCFB(str, key, iv) + plainText, err = encrypt.AESDecryptWithCFB([]byte(strBuf.GetString(i)), key, iv) } if err != nil { result.AppendNull() @@ -487,20 +422,14 @@ func (b *builtinMD5Sig) vecEvalString(input *chunk.Chunk, result *chunk.Column) } result.ReserveString(n) - var dBytes []byte digest := md5.New() // #nosec G401 - enc := charset.NewEncoding(b.args[0].GetType().Charset) for i := 0; i < n; i++ { if buf.IsNull(i) { result.AppendNull() continue } cryptBytes := buf.GetBytes(i) - dBytes, err := enc.Encode(dBytes, cryptBytes) - if err != nil { - return err - } - _, err = digest.Write(dBytes) + _, err = digest.Write(cryptBytes) if err != nil { return err } @@ -625,9 +554,6 @@ 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++ { @@ -636,19 +562,14 @@ func (b *builtinCompressSig) vecEvalString(input *chunk.Chunk, result *chunk.Col continue } - str := buf.GetBytes(i) + strBytes := buf.GetBytes(i) // According to doc: Empty strings are stored as empty strings. - if len(str) == 0 { + if len(strBytes) == 0 { result.AppendString("") } - strBuf, err := bufEnc.Encode(encodedBuf, str) - if err != nil { - return err - } - - compressed, err := deflate(strBuf) + compressed, err := deflate(strBytes) if err != nil { result.AppendNull() continue @@ -666,7 +587,7 @@ func (b *builtinCompressSig) vecEvalString(input *chunk.Chunk, result *chunk.Col defer deallocateByteSlice(buffer) buffer = buffer[:resultLength] - binary.LittleEndian.PutUint32(buffer, uint32(len(strBuf))) + binary.LittleEndian.PutUint32(buffer, uint32(len(strBytes))) copy(buffer[4:], compressed) if shouldAppendSuffix { @@ -694,7 +615,6 @@ func (b *builtinAesEncryptSig) vecEvalString(input *chunk.Chunk, result *chunk.C if err := b.args[0].VecEvalString(b.ctx, input, strBuf); err != nil { return err } - enc := charset.NewEncoding(b.args[0].GetType().Charset) keyBuf, err := b.bufAllocator.get() if err != nil { @@ -712,7 +632,7 @@ func (b *builtinAesEncryptSig) vecEvalString(input *chunk.Chunk, result *chunk.C isWarning := !b.ivRequired && len(b.args) == 3 isConst := b.args[1].ConstItem(b.ctx.GetSessionVars().StmtCtx) - var key, dBytes []byte + var key []byte if isConst { key = encrypt.DeriveKeyMySQL(keyBuf.GetBytes(0), b.keySize) } @@ -735,10 +655,6 @@ func (b *builtinAesEncryptSig) vecEvalString(input *chunk.Chunk, result *chunk.C // NOTE: we can't use GetBytes, because in AESEncryptWithECB padding is automatically // added to str and this will damange the data layout in chunk.Column str := []byte(strBuf.GetString(i)) - str, err := enc.Encode(dBytes, str) - if err != nil { - return err - } cipherText, err := encrypt.AESEncryptWithECB(str, key) if err != nil { result.AppendNull() @@ -764,9 +680,6 @@ func (b *builtinPasswordSig) vecEvalString(input *chunk.Chunk, result *chunk.Col if err := b.args[0].VecEvalString(b.ctx, input, buf); err != nil { return err } - - var dBytes []byte - enc := charset.NewEncoding(b.args[0].GetType().Charset) result.ReserveString(n) for i := 0; i < n; i++ { if buf.IsNull(i) { @@ -780,16 +693,11 @@ func (b *builtinPasswordSig) vecEvalString(input *chunk.Chunk, result *chunk.Col continue } - dBytes, err := enc.Encode(dBytes, passBytes) - if err != nil { - return err - } - // We should append a warning here because function "PASSWORD" is deprecated since MySQL 5.7.6. // See https://dev.mysql.com/doc/refman/5.7/en/encryption-functions.html#function_password b.ctx.GetSessionVars().StmtCtx.AppendWarning(errDeprecatedSyntaxNoReplacement.GenWithStackByArgs("PASSWORD")) - result.AppendString(auth.EncodePasswordBytes(dBytes)) + result.AppendString(auth.EncodePasswordBytes(passBytes)) } return nil } @@ -808,8 +716,6 @@ func (b *builtinSHA1Sig) vecEvalString(input *chunk.Chunk, result *chunk.Column) if err := b.args[0].VecEvalString(b.ctx, input, buf); err != nil { return err } - var dBytes []byte - enc := charset.NewEncoding(b.args[0].GetType().Charset) result.ReserveString(n) hasher := sha1.New() // #nosec G401 for i := 0; i < n; i++ { @@ -818,10 +724,6 @@ func (b *builtinSHA1Sig) vecEvalString(input *chunk.Chunk, result *chunk.Column) continue } str := buf.GetBytes(i) - str, err := enc.Encode(dBytes, str) - if err != nil { - return err - } _, err = hasher.Write(str) if err != nil { return err diff --git a/expression/builtin_math.go b/expression/builtin_math.go index d179ddcfc0bc8..54fae95639142 100644 --- a/expression/builtin_math.go +++ b/expression/builtin_math.go @@ -24,7 +24,6 @@ import ( "math" "strconv" "strings" - "sync" "github.com/cznic/mathutil" "github.com/pingcap/tidb/parser/ast" @@ -32,6 +31,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + utilMath "github.com/pingcap/tidb/util/math" "github.com/pingcap/tipb/go-tipb" ) @@ -273,14 +273,17 @@ func (c *roundFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi bf.tp.Flag |= mysql.UnsignedFlag } - bf.tp.Flen = argFieldTp.Flen - bf.tp.Decimal = calculateDecimal4RoundAndTruncate(ctx, args, argTp) - if bf.tp.Decimal != types.UnspecifiedLength { - if argFieldTp.Decimal != types.UnspecifiedLength { - decimalDelta := bf.tp.Decimal - argFieldTp.Decimal - bf.tp.Flen += mathutil.Max(decimalDelta, 0) - } else { - bf.tp.Flen = argFieldTp.Flen + bf.tp.Decimal + // ETInt or ETReal is set correctly by newBaseBuiltinFuncWithTp, only need to handle ETDecimal. + if argTp == types.ETDecimal { + bf.tp.Flen = argFieldTp.Flen + bf.tp.Decimal = calculateDecimal4RoundAndTruncate(ctx, args, argTp) + if bf.tp.Decimal != types.UnspecifiedLength { + if argFieldTp.Decimal != types.UnspecifiedLength { + decimalDelta := bf.tp.Decimal - argFieldTp.Decimal + bf.tp.Flen += mathutil.Max(decimalDelta, 0) + } else { + bf.tp.Flen = argFieldTp.Flen + bf.tp.Decimal + } } } @@ -487,8 +490,10 @@ func (c *ceilFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio return nil, err } setFlag4FloorAndCeil(bf.tp, args[0]) - argFieldTp := args[0].GetType() - bf.tp.Flen, bf.tp.Decimal = argFieldTp.Flen, 0 + // ETInt or ETReal is set correctly by newBaseBuiltinFuncWithTp, only need to handle ETDecimal. + if retTp == types.ETDecimal { + bf.tp.Flen, bf.tp.Decimal = args[0].GetType().Flen, 0 + } switch argTp { case types.ETInt: @@ -675,7 +680,11 @@ func (c *floorFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi return nil, err } setFlag4FloorAndCeil(bf.tp, args[0]) - bf.tp.Flen, bf.tp.Decimal = args[0].GetType().Flen, 0 + + // ETInt or ETReal is set correctly by newBaseBuiltinFuncWithTp, only need to handle ETDecimal. + if retTp == types.ETDecimal { + bf.tp.Flen, bf.tp.Decimal = args[0].GetType().Flen, 0 + } switch argTp { case types.ETInt: if retTp == types.ETInt { @@ -1014,7 +1023,7 @@ func (c *randFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio } bt := bf if len(args) == 0 { - sig = &builtinRandSig{bt, &sync.Mutex{}, NewWithTime()} + sig = &builtinRandSig{bt, ctx.GetSessionVars().Rng} sig.setPbCode(tipb.ScalarFuncSig_Rand) } else if _, isConstant := args[0].(*Constant); isConstant { // According to MySQL manual: @@ -1030,7 +1039,7 @@ func (c *randFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio // The behavior same as MySQL. seed = 0 } - sig = &builtinRandSig{bt, &sync.Mutex{}, NewWithSeed(seed)} + sig = &builtinRandSig{bt, utilMath.NewWithSeed(seed)} sig.setPbCode(tipb.ScalarFuncSig_Rand) } else { sig = &builtinRandWithSeedFirstGenSig{bt} @@ -1041,12 +1050,11 @@ func (c *randFunctionClass) getFunction(ctx sessionctx.Context, args []Expressio type builtinRandSig struct { baseBuiltinFunc - mu *sync.Mutex - mysqlRng *MysqlRng + mysqlRng *utilMath.MysqlRng } func (b *builtinRandSig) Clone() builtinFunc { - newSig := &builtinRandSig{mysqlRng: b.mysqlRng, mu: b.mu} + newSig := &builtinRandSig{mysqlRng: b.mysqlRng} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -1054,9 +1062,7 @@ func (b *builtinRandSig) Clone() builtinFunc { // evalReal evals RAND(). // See https://dev.mysql.com/doc/refman/5.7/en/mathematical-functions.html#function_rand func (b *builtinRandSig) evalReal(row chunk.Row) (float64, bool, error) { - b.mu.Lock() res := b.mysqlRng.Gen() - b.mu.Unlock() return res, false, nil } @@ -1080,11 +1086,11 @@ func (b *builtinRandWithSeedFirstGenSig) evalReal(row chunk.Row) (float64, bool, // b.args[0] is promised to be a non-constant(such as a column name) in // builtinRandWithSeedFirstGenSig, the seed is initialized with the value for each // invocation of RAND(). - var rng *MysqlRng + var rng *utilMath.MysqlRng if !isNull { - rng = NewWithSeed(seed) + rng = utilMath.NewWithSeed(seed) } else { - rng = NewWithSeed(0) + rng = utilMath.NewWithSeed(0) } return rng.Gen(), false, nil } @@ -1898,9 +1904,11 @@ func (c *truncateFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err != nil { return nil, err } - - bf.tp.Decimal = calculateDecimal4RoundAndTruncate(ctx, args, argTp) - bf.tp.Flen = args[0].GetType().Flen - args[0].GetType().Decimal + bf.tp.Decimal + // ETInt or ETReal is set correctly by newBaseBuiltinFuncWithTp, only need to handle ETDecimal. + if argTp == types.ETDecimal { + bf.tp.Decimal = calculateDecimal4RoundAndTruncate(ctx, args, argTp) + bf.tp.Flen = args[0].GetType().Flen - args[0].GetType().Decimal + bf.tp.Decimal + } bf.tp.Flag |= args[0].GetType().Flag var sig builtinFunc diff --git a/expression/builtin_math_test.go b/expression/builtin_math_test.go index 5a3d5f4cd6068..88d2985f10c17 100644 --- a/expression/builtin_math_test.go +++ b/expression/builtin_math_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/testkit/trequire" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + utilMath "github.com/pingcap/tidb/util/math" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" ) @@ -382,7 +383,7 @@ func TestRand(t *testing.T) { // issue 3211 f2, err := fc.getFunction(ctx, []Expression{&Constant{Value: types.NewIntDatum(20160101), RetType: types.NewFieldType(mysql.TypeLonglong)}}) require.NoError(t, err) - randGen := NewWithSeed(20160101) + randGen := utilMath.NewWithSeed(20160101) for i := 0; i < 3; i++ { v, err = evalBuiltinFunc(f2, chunk.Row{}) require.NoError(t, err) diff --git a/expression/builtin_math_vec.go b/expression/builtin_math_vec.go index de1b6886d51cc..13b6a701d5d8e 100644 --- a/expression/builtin_math_vec.go +++ b/expression/builtin_math_vec.go @@ -20,6 +20,8 @@ import ( "math" "strconv" + utilMath "github.com/pingcap/tidb/util/math" + "github.com/cznic/mathutil" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" @@ -709,11 +711,9 @@ func (b *builtinRandSig) vecEvalReal(input *chunk.Chunk, result *chunk.Column) e n := input.NumRows() result.ResizeFloat64(n, false) f64s := result.Float64s() - b.mu.Lock() for i := range f64s { f64s[i] = b.mysqlRng.Gen() } - b.mu.Unlock() return nil } @@ -738,9 +738,9 @@ func (b *builtinRandWithSeedFirstGenSig) vecEvalReal(input *chunk.Chunk, result for i := 0; i < n; i++ { // When the seed is null we need to use 0 as the seed. // The behavior same as MySQL. - rng := NewWithSeed(0) + rng := utilMath.NewWithSeed(0) if !buf.IsNull(i) { - rng = NewWithSeed(i64s[i]) + rng = utilMath.NewWithSeed(i64s[i]) } f64s[i] = rng.Gen() } diff --git a/expression/builtin_other_test.go b/expression/builtin_other_test.go index 7979f7d2720a6..58263af992365 100644 --- a/expression/builtin_other_test.go +++ b/expression/builtin_other_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/stretchr/testify/require" ) @@ -198,7 +199,7 @@ func TestValues(t *testing.T) { ret, err = evalBuiltinFunc(sig, chunk.Row{}) require.NoError(t, err) - cmp, err := ret.CompareDatum(nil, &currInsertValues[1]) + cmp, err := ret.Compare(nil, &currInsertValues[1], collate.GetBinaryCollator()) require.NoError(t, err) require.Equal(t, 0, cmp) } diff --git a/expression/builtin_string.go b/expression/builtin_string.go index bf948605045b2..38a5647d49fac 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -2333,8 +2333,12 @@ func (b *builtinBitLengthSig) evalInt(row chunk.Row) (int64, bool, error) { if isNull || err != nil { return 0, isNull, err } - - return int64(len(val) * 8), false, nil + argTp := b.args[0].GetType() + dBytes, err := charset.NewEncoding(argTp.Charset).Encode(nil, hack.Slice(val)) + if err != nil { + return 0, isNull, err + } + return int64(len(dBytes) * 8), false, nil } type charFunctionClass struct { @@ -2895,14 +2899,21 @@ func (b *builtinOrdSig) evalInt(row chunk.Row) (int64, bool, error) { return 0, isNull, err } - ord, err := chooseOrdFunc(b.args[0].GetType().Charset) + charSet := b.args[0].GetType().Charset + ord, err := chooseOrdFunc(charSet) + if err != nil { + return 0, false, err + } + + enc := charset.NewEncoding(charSet) + leftMost, err := enc.EncodeFirstChar(nil, hack.Slice(str)) if err != nil { return 0, false, err } - return ord(str), false, nil + return ord(leftMost), false, nil } -func chooseOrdFunc(charSet string) (func(string) int64, error) { +func chooseOrdFunc(charSet string) (func([]byte) int64, error) { // use utf8 by default if charSet == "" { charSet = charset.CharsetUTF8 @@ -2914,22 +2925,17 @@ func chooseOrdFunc(charSet string) (func(string) int64, error) { if desc.Maxlen == 1 { return ordSingleByte, nil } - return ordUtf8, nil + return ordOthers, nil } -func ordSingleByte(str string) int64 { - if len(str) == 0 { +func ordSingleByte(src []byte) int64 { + if len(src) == 0 { return 0 } - return int64(str[0]) + return int64(src[0]) } -func ordUtf8(str string) int64 { - if len(str) == 0 { - return 0 - } - _, size := utf8.DecodeRuneInString(str) - leftMost := str[:size] +func ordOthers(leftMost []byte) int64 { var result int64 var factor int64 = 1 for i := len(leftMost) - 1; i >= 0; i-- { @@ -3666,7 +3672,7 @@ func (c *insertFunctionClass) getFunction(ctx sessionctx.Context, args []Express return nil, errors.Trace(err) } - if types.IsBinaryStr(args[0].GetType()) { + if types.IsBinaryStr(bf.tp) { sig = &builtinInsertSig{bf, maxAllowedPacket} sig.setPbCode(tipb.ScalarFuncSig_Insert) } else { diff --git a/expression/builtin_string_test.go b/expression/builtin_string_test.go index 28d98d0215091..6035e8b120fbf 100644 --- a/expression/builtin_string_test.go +++ b/expression/builtin_string_test.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/testkit/trequire" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" ) @@ -1390,16 +1391,24 @@ func TestBitLength(t *testing.T) { ctx := createContext(t) cases := []struct { args interface{} + chs string expected int64 isNil bool getErr bool }{ - {"hi", 16, false, false}, - {"你好", 48, false, false}, - {"", 0, false, false}, + {"hi", "", 16, false, false}, + {"你好", "", 48, false, false}, + {"", "", 0, false, false}, + {"abc", "gbk", 24, false, false}, + {"一二三", "gbk", 48, false, false}, + {"一二三", "", 72, false, false}, + {"一二三!", "gbk", 56, false, false}, + {"一二三!", "", 80, false, false}, } for _, c := range cases { + err := ctx.GetSessionVars().SetSystemVar(variable.CharacterSetConnection, c.chs) + require.NoError(t, err) f, err := newFunctionForTest(ctx, ast.BitLength, primitiveValsToConstants(ctx, []interface{}{c.args})...) require.NoError(t, err) d, err := f.Eval(chunk.Row{}) @@ -2221,28 +2230,37 @@ func TestInsert(t *testing.T) { } func TestOrd(t *testing.T) { - t.Parallel() + // TODO: Remove this and enable test parallel after new charset enabled + collate.SetCharsetFeatEnabledForTest(true) + defer collate.SetCharsetFeatEnabledForTest(false) ctx := createContext(t) cases := []struct { args interface{} expected int64 + chs string isNil bool getErr bool }{ - {"2", 50, false, false}, - {2, 50, false, false}, - {"23", 50, false, false}, - {23, 50, false, false}, - {2.3, 50, false, false}, - {nil, 0, true, false}, - {"", 0, false, false}, - {"你好", 14990752, false, false}, - {"にほん", 14909867, false, false}, - {"한국", 15570332, false, false}, - {"👍", 4036989325, false, false}, - {"א", 55184, false, false}, + {"2", 50, "", false, false}, + {2, 50, "", false, false}, + {"23", 50, "", false, false}, + {23, 50, "", false, false}, + {2.3, 50, "", false, false}, + {nil, 0, "", true, false}, + {"", 0, "", false, false}, + {"你好", 14990752, "", false, false}, + {"にほん", 14909867, "", false, false}, + {"한국", 15570332, "", false, false}, + {"👍", 4036989325, "", false, false}, + {"א", 55184, "", false, false}, + {"abc", 97, "gbk", false, false}, + {"一二三", 53947, "gbk", false, false}, + {"àáèé", 43172, "gbk", false, false}, + {"数据库", 51965, "gbk", false, false}, } for _, c := range cases { + err := ctx.GetSessionVars().SetSystemVar(variable.CharacterSetConnection, c.chs) + require.NoError(t, err) f, err := newFunctionForTest(ctx, ast.Ord, primitiveValsToConstants(ctx, []interface{}{c.args})...) require.NoError(t, err) diff --git a/expression/builtin_string_vec.go b/expression/builtin_string_vec.go index 11933f305fe58..34293d6103be0 100644 --- a/expression/builtin_string_vec.go +++ b/expression/builtin_string_vec.go @@ -2070,11 +2070,15 @@ func (b *builtinOrdSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) err return err } - ord, err := chooseOrdFunc(b.args[0].GetType().Charset) + charSet := b.args[0].GetType().Charset + ord, err := chooseOrdFunc(charSet) if err != nil { return err } + enc := charset.NewEncoding(charSet) + var encodedBuf []byte + result.ResizeInt64(n, false) result.MergeNulls(buf) i64s := result.Int64s() @@ -2082,8 +2086,12 @@ func (b *builtinOrdSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) err if result.IsNull(i) { continue } - str := buf.GetString(i) - i64s[i] = ord(str) + str := buf.GetBytes(i) + encoded, err := enc.EncodeFirstChar(encodedBuf, str) + if err != nil { + return err + } + i64s[i] = ord(encoded) } return nil } @@ -2225,6 +2233,9 @@ func (b *builtinBitLengthSig) vecEvalInt(input *chunk.Chunk, result *chunk.Colum return err } + argTp := b.args[0].GetType() + enc := charset.NewEncoding(argTp.Charset) + result.ResizeInt64(n, false) result.MergeNulls(buf) i64s := result.Int64s() @@ -2233,7 +2244,11 @@ func (b *builtinBitLengthSig) vecEvalInt(input *chunk.Chunk, result *chunk.Colum continue } str := buf.GetBytes(i) - i64s[i] = int64(len(str) * 8) + dBytes, err := enc.Encode(nil, str) + if err != nil { + return err + } + i64s[i] = int64(len(dBytes) * 8) } return nil } diff --git a/expression/collation.go b/expression/collation.go index 7db5645941601..80a2720c8cfe4 100644 --- a/expression/collation.go +++ b/expression/collation.go @@ -301,7 +301,8 @@ func safeConvert(ctx sessionctx.Context, ec *ExprCollation, args ...Expression) continue } - if arg.Repertoire() == ASCII { + // If value has ASCII repertoire, or it is binary string, just skip it. + if arg.Repertoire() == ASCII || types.IsBinaryStr(arg.GetType()) { continue } @@ -327,12 +328,7 @@ func safeConvert(ctx sessionctx.Context, ec *ExprCollation, args ...Expression) func isValidString(str string, dstChs string) bool { switch dstChs { case charset.CharsetASCII: - for _, c := range str { - if c >= 0x80 { - return false - } - } - return true + return charset.StringValidatorASCII{}.Validate(str) == -1 case charset.CharsetLatin1: // For backward compatibility, we do not block SQL like select '啊' = convert('a' using latin1) collate latin1_bin; return true @@ -343,9 +339,7 @@ func isValidString(str string, dstChs string) bool { // Convert to binary is always safe. return true default: - e, _ := charset.Lookup(dstChs) - _, err := e.NewEncoder().String(str) - return err == nil + return charset.StringValidatorOther{Charset: dstChs}.Validate(str) == -1 } } diff --git a/expression/constant_test.go b/expression/constant_test.go index 2158b1e4f5d66..fc46c40f1072d 100644 --- a/expression/constant_test.go +++ b/expression/constant_test.go @@ -57,12 +57,11 @@ func newString(value string, collation string) *Constant { } func newFunction(funcName string, args ...Expression) Expression { - return newFunctionWithType(funcName, mysql.TypeLonglong, args...) + return newFunctionWithType(funcName, types.NewFieldType(mysql.TypeLonglong), args...) } -func newFunctionWithType(funcName string, tp byte, args ...Expression) Expression { - ft := types.NewFieldType(tp) - return NewFunctionInternal(mock.NewContext(), funcName, ft, args...) +func newFunctionWithType(funcName string, tp *types.FieldType, args ...Expression) Expression { + return NewFunctionInternal(mock.NewContext(), funcName, tp, args...) } func TestConstantPropagation(t *testing.T) { @@ -239,16 +238,48 @@ func TestConstantFoldingCharsetConvert(t *testing.T) { }{ { condition: newFunction(ast.Length, newFunctionWithType( - InternalFuncToBinary, mysql.TypeVarchar, + InternalFuncToBinary, types.NewFieldType(mysql.TypeVarchar), newString("中文", "gbk_bin"))), result: "4", }, { condition: newFunction(ast.Length, newFunctionWithType( - InternalFuncToBinary, mysql.TypeVarchar, + InternalFuncToBinary, types.NewFieldType(mysql.TypeVarchar), newString("中文", "utf8mb4_bin"))), result: "6", }, + { + condition: newFunction(ast.Concat, newFunctionWithType( + InternalFuncFromBinary, types.NewFieldType(mysql.TypeVarchar), + newString("中文", "binary"))), + result: "中文", + }, + { + condition: newFunction(ast.Concat, + newFunctionWithType( + InternalFuncFromBinary, types.NewFieldTypeWithCollation(mysql.TypeVarchar, "gbk_bin", -1), + newString("\xd2\xbb", "binary")), + newString("中文", "gbk_bin"), + ), + result: "一中文", + }, + { + condition: newFunction(ast.Concat, + newString("中文", "gbk_bin"), + newFunctionWithType( + InternalFuncFromBinary, types.NewFieldTypeWithCollation(mysql.TypeVarchar, "gbk_bin", -1), + newString("\xd2\xbb", "binary")), + ), + result: "中文一", + }, + // The result is binary charset, so gbk constant will convert to binary which is \xd6\xd0\xce\xc4. + { + condition: newFunction(ast.Concat, + newString("中文", "gbk_bin"), + newString("\xd2\xbb", "binary"), + ), + result: "\xd6\xd0\xce\xc4\xd2\xbb", + }, } for _, tt := range tests { newConds := FoldConstant(tt.condition) diff --git a/expression/distsql_builtin.go b/expression/distsql_builtin.go index db9b39a2db010..1763f71d2fb05 100644 --- a/expression/distsql_builtin.go +++ b/expression/distsql_builtin.go @@ -1049,6 +1049,10 @@ func getSignatureByPB(ctx sessionctx.Context, sigCode tipb.ScalarFuncSig, tp *ti f = &builtinUnHexSig{base} case tipb.ScalarFuncSig_Upper: f = &builtinUpperSig{base} + case tipb.ScalarFuncSig_ToBinary: + f = &builtinInternalToBinarySig{base} + case tipb.ScalarFuncSig_FromBinary: + f = &builtinInternalFromBinarySig{base} default: e = errFunctionNotExists.GenWithStackByArgs("FUNCTION", sigCode) diff --git a/expression/distsql_builtin_test.go b/expression/distsql_builtin_test.go index c08c728d63446..06ced0d27211f 100644 --- a/expression/distsql_builtin_test.go +++ b/expression/distsql_builtin_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" ) @@ -787,7 +788,7 @@ func TestEval(t *testing.T) { result, err := expr.Eval(row) require.NoError(t, err) require.Equal(t, tt.result.Kind(), result.Kind()) - cmp, err := result.CompareDatum(sc, &tt.result) + cmp, err := result.Compare(sc, &tt.result, collate.GetCollator(fieldTps[0].Collate)) require.NoError(t, err) require.Equal(t, 0, cmp) } diff --git a/expression/expression.go b/expression/expression.go index fda1615fd8b4e..e4d589f796194 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -967,6 +967,7 @@ func scalarExprSupportedByTiKV(sf *ScalarFunction) bool { // string functions. ast.Length, ast.BitLength, ast.Concat, ast.ConcatWS /*ast.Locate,*/, ast.Replace, ast.ASCII, ast.Hex, ast.Reverse, ast.LTrim, ast.RTrim /*ast.Left,*/, ast.Strcmp, ast.Space, ast.Elt, ast.Field, + InternalFuncFromBinary, InternalFuncToBinary, // json functions. ast.JSONType, ast.JSONExtract, ast.JSONObject, ast.JSONArray, ast.JSONMerge, ast.JSONSet, diff --git a/expression/integration_test.go b/expression/integration_test.go index 1d998efbc82e1..eca90299fd12b 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -794,6 +794,10 @@ func (s *testIntegrationSuite2) TestMathBuiltin(c *C) { tk.MustQuery("select rand(1) from t").Sort().Check(testkit.Rows("0.1418603212962489", "0.40540353712197724", "0.8716141803857071")) tk.MustQuery("select rand(a) from t").Check(testkit.Rows("0.40540353712197724", "0.6555866465490187", "0.9057697559760601")) tk.MustQuery("select rand(1), rand(2), rand(3)").Check(testkit.Rows("0.40540353712197724 0.6555866465490187 0.9057697559760601")) + tk.MustQuery("set @@rand_seed1=10000000,@@rand_seed2=1000000") + tk.MustQuery("select rand()").Check(testkit.Rows("0.028870999839968048")) + tk.MustQuery("select rand(1)").Check(testkit.Rows("0.40540353712197724")) + tk.MustQuery("select rand()").Check(testkit.Rows("0.11641535266900002")) } func (s *testIntegrationSuite2) TestStringBuiltin(c *C) { @@ -1180,7 +1184,7 @@ func (s *testIntegrationSuite2) TestStringBuiltin(c *C) { // for insert result = tk.MustQuery(`select insert("中文", 1, 1, cast("aaa" as binary)), insert("ba", -1, 1, "aaa"), insert("ba", 1, 100, "aaa"), insert("ba", 100, 1, "aaa");`) - result.Check(testkit.Rows("aaa文 ba aaa ba")) + result.Check(testkit.Rows("aaa\xb8\xad文 ba aaa ba")) result = tk.MustQuery(`select insert("bb", NULL, 1, "aa"), insert("bb", 1, NULL, "aa"), insert(NULL, 1, 1, "aaa"), insert("bb", 1, 1, NULL);`) result.Check(testkit.Rows(" ")) result = tk.MustQuery(`SELECT INSERT("bb", 0, 1, NULL), INSERT("bb", 0, NULL, "aaa");`) @@ -5202,7 +5206,7 @@ func (s *testIntegrationSuite) TestUnknowHintIgnore(c *C) { tk.MustExec("USE test") tk.MustExec("create table t(a int)") tk.MustQuery("select /*+ unknown_hint(c1)*/ 1").Check(testkit.Rows("1")) - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1064 You have an error in your SQL syntax; check the manual that corresponds to your TiDB version for the right syntax to use [parser:8064]Optimizer hint syntax error at line 1 column 23 near \"unknown_hint(c1)*/\" ")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1064 Optimizer hint syntax error at line 1 column 23 near \"unknown_hint(c1)*/\" ")) _, err := tk.Exec("select 1 from /*+ test1() */ t") c.Assert(err, IsNil) } @@ -6207,6 +6211,39 @@ func (s *testIntegrationSerialSuite) TestPreparePlanCache(c *C) { tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) } +func (s *testIntegrationSerialSuite) TestPreparePlanCacheNotForCacheTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + + plannercore.SetPreparedPlanCache(true) + c.Assert(plannercore.PreparedPlanCacheEnabled(), Equals, true) + var err error + tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{ + PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), + }) + c.Assert(err, IsNil) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int);") + tk.MustExec("alter table t cache") + + tk.MustQuery("select * from t where a = 1") + // already read cache after reading first time + tk.MustQuery("explain format = 'brief' select * from t where a = 1").Check(testkit.Rows( + "Projection 10.00 root test.t.a", + "└─UnionScan 10.00 root eq(test.t.a, 1)", + " └─TableReader 10.00 root data:Selection", + " └─Selection 10.00 cop[tikv] eq(test.t.a, 1)", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo")) + + tk.MustExec("prepare stmt from 'select * from t where a = ?';") + tk.MustExec("set @a = 1;") + tk.MustExec("execute stmt using @a;") + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) + tk.MustExec("execute stmt using @a;") + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) +} + func (s *testIntegrationSerialSuite) TestIssue16205(c *C) { tk := testkit.NewTestKit(c, s.store) orgEnable := plannercore.PreparedPlanCacheEnabled() @@ -9503,6 +9540,11 @@ func (s *testIntegrationSuite) TestEnumPushDown(c *C) { tk.MustExec("insert into tdm values (1, 'a');") tk.MustExec("update tdm set c12 = 2 where id = 1;") tk.MustQuery("select * from tdm").Check(testkit.Rows("1 b")) + tk.MustExec("set @@sql_mode = '';") + tk.MustExec("update tdm set c12 = 0 where id = 1;") + tk.MustQuery("select c12+0 from tdm").Check(testkit.Rows("0")) + tk.MustExec("update tdm set c12 = '0' where id = 1;") + tk.MustQuery("select c12+0 from tdm").Check(testkit.Rows("0")) } func (s *testIntegrationSuite) TestJiraSetInnoDBDefaultRowFormat(c *C) { @@ -10568,6 +10610,15 @@ func (s *testIntegrationSuite) TestIssue29434(c *C) { tk.MustQuery("select least(c1, '99999999999999') from t1;").Check(testkit.Rows("2021-12-12 10:10:10")) } +func (s *testIntegrationSuite) TestIssue29417(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1 (f1 decimal(5,5));") + tk.MustExec("insert into t1 values (-0.12345);") + tk.MustQuery("select concat(f1) from t1;").Check(testkit.Rows("-0.12345")) +} + func (s *testIntegrationSuite) TestIssue29244(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -10580,3 +10631,16 @@ func (s *testIntegrationSuite) TestIssue29244(c *C) { tk.MustExec("set tidb_enable_vectorized_expression = off;") tk.MustQuery("select microsecond(a) from t;").Check(testkit.Rows("123500", "123500")) } + +func (s *testIntegrationSuite) TestIssue29513(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustQuery("select '123' union select cast(45678 as char);").Sort().Check(testkit.Rows("123", "45678")) + tk.MustQuery("select '123' union select cast(45678 as char(2));").Sort().Check(testkit.Rows("123", "45")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t values(45678);") + tk.MustQuery("select '123' union select cast(a as char) from t;").Sort().Check(testkit.Rows("123", "45678")) + tk.MustQuery("select '123' union select cast(a as char(2)) from t;").Sort().Check(testkit.Rows("123", "45")) +} diff --git a/expression/main_test.go b/expression/main_test.go index 1c68ba643159b..84a4ad15392ad 100644 --- a/expression/main_test.go +++ b/expression/main_test.go @@ -48,6 +48,7 @@ func TestMain(m *testing.M) { opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + goleak.IgnoreTopFunction("github.com/pingcap/tidb/table/tables.mockRemoteService"), } goleak.VerifyTestMain(m, opts...) diff --git a/expression/scalar_function.go b/expression/scalar_function.go index 220eeb090a5b1..2030186a53176 100644 --- a/expression/scalar_function.go +++ b/expression/scalar_function.go @@ -183,6 +183,10 @@ func newFunctionImpl(ctx sessionctx.Context, fold int, funcName string, retType return BuildCastFunction(ctx, args[0], retType), nil case ast.GetVar: return BuildGetVarFunction(ctx, args[0], retType) + case InternalFuncFromBinary: + return BuildFromBinaryFunction(ctx, args[0], retType), nil + case InternalFuncToBinary: + return BuildToBinaryFunction(ctx, args[0]), nil } fc, ok := funcs[funcName] if !ok { @@ -364,6 +368,14 @@ func (sf *ScalarFunction) Eval(row chunk.Row) (d types.Datum, err error) { str, isNull, err = sf.EvalString(sf.GetCtx(), row) if !isNull && err == nil && tp.Tp == mysql.TypeEnum { res, err = types.ParseEnum(tp.Elems, str, tp.Collate) + if ctx := sf.GetCtx(); ctx != nil { + if sc := ctx.GetSessionVars().StmtCtx; sc != nil { + if sc.TruncateAsWarning { + ctx.GetSessionVars().StmtCtx.AppendWarning(err) + err = nil + } + } + } } else { res = str } diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index f2e94aa7c093e..8cba8c5ef6b8f 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -181,9 +181,9 @@ func (s *InferTypeSuite) createTestCase4Constants() []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)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 11, -1}, {"CAST(c_int_d AS BINARY(5))", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 5, -1}, - {"CAST(c_int_d AS CHAR)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, -1, -1}, // TODO: Flen should be 11. + {"CAST(c_int_d AS CHAR)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 11, -1}, {"CAST(c_int_d AS CHAR(5))", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 5, -1}, {"CAST(c_int_d AS DATE)", mysql.TypeDate, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, {"CAST(c_int_d AS DATETIME)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 19, 0}, @@ -329,7 +329,7 @@ func (s *InferTypeSuite) createTestCase4StrFuncs() []typeInferTestCase { {"from_base64(c_bigint_d )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 60, types.UnspecifiedLength}, {"from_base64(c_float_d )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, types.UnspecifiedLength, types.UnspecifiedLength}, {"from_base64(c_double_d )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, types.UnspecifiedLength, types.UnspecifiedLength}, - {"from_base64(c_decimal )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 24, types.UnspecifiedLength}, + {"from_base64(c_decimal )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 27, types.UnspecifiedLength}, {"from_base64(c_datetime )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 66, types.UnspecifiedLength}, {"from_base64(c_time_d )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 30, types.UnspecifiedLength}, {"from_base64(c_timestamp_d)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 57, types.UnspecifiedLength}, @@ -430,9 +430,9 @@ func (s *InferTypeSuite) createTestCase4StrFuncs() []typeInferTestCase { {"reverse(c_int_d )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 20, types.UnspecifiedLength}, {"reverse(c_bigint_d )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 20, types.UnspecifiedLength}, - {"reverse(c_float_d )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, -1, types.UnspecifiedLength}, - {"reverse(c_double_d )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, -1, types.UnspecifiedLength}, - {"reverse(c_decimal )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 8, types.UnspecifiedLength}, + {"reverse(c_float_d )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, types.UnspecifiedLength, types.UnspecifiedLength}, + {"reverse(c_double_d )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, types.UnspecifiedLength, types.UnspecifiedLength}, + {"reverse(c_decimal )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 9, types.UnspecifiedLength}, {"reverse(c_char )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 20, types.UnspecifiedLength}, {"reverse(c_varchar )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 20, types.UnspecifiedLength}, {"reverse(c_text_d )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 65535, types.UnspecifiedLength}, @@ -547,51 +547,51 @@ func (s *InferTypeSuite) createTestCase4MathFuncs() []typeInferTestCase { {"cot(c_timestamp_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, {"cot(c_binary)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, - {"floor(c_int_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 11, 0}, - {"floor(c_uint_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.UnsignedFlag | mysql.BinaryFlag, 10, 0}, - {"floor(c_bigint_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 20, 0}, // TODO: Flen should be 17 - {"floor(c_ubigint_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.UnsignedFlag | mysql.BinaryFlag, 20, 0}, // TODO: Flen should be 17 - {"floor(c_decimal)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 6, 0}, - {"floor(c_udecimal)", mysql.TypeLonglong, charset.CharsetBin, mysql.UnsignedFlag | mysql.BinaryFlag, 10, 0}, - {"floor(c_double_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 22, 0}, - {"floor(c_udouble_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 22, 0}, - {"floor(c_float_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 12, 0}, - {"floor(c_ufloat_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 12, 0}, - {"floor(c_datetime)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"floor(c_timestamp_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"floor(c_time_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"floor(c_enum)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"floor(c_text_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"floor(18446744073709551615)", mysql.TypeLonglong, charset.CharsetBin, mysql.NotNullFlag | mysql.UnsignedFlag | mysql.BinaryFlag, 20, 0}, + {"floor(c_int_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"floor(c_uint_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.UnsignedFlag | mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"floor(c_bigint_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"floor(c_ubigint_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.UnsignedFlag | mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"floor(c_decimal)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"floor(c_udecimal)", mysql.TypeLonglong, charset.CharsetBin, mysql.UnsignedFlag | mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"floor(c_double_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"floor(c_udouble_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"floor(c_float_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"floor(c_ufloat_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"floor(c_datetime)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"floor(c_timestamp_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"floor(c_time_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"floor(c_enum)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"floor(c_text_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"floor(18446744073709551615)", mysql.TypeLonglong, charset.CharsetBin, mysql.NotNullFlag | mysql.UnsignedFlag | mysql.BinaryFlag, mysql.MaxIntWidth, 0}, {"floor(18446744073709551615.1)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 23, 0}, - {"ceil(c_int_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 11, 0}, - {"ceil(c_uint_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.UnsignedFlag | mysql.BinaryFlag, 10, 0}, - {"ceil(c_bigint_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 20, 0}, // TODO: Flen should be 17 - {"ceil(c_ubigint_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.UnsignedFlag | mysql.BinaryFlag, 20, 0}, // TODO: Flen should be 17 - {"ceil(c_decimal)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 6, 0}, - {"floor(c_udecimal)", mysql.TypeLonglong, charset.CharsetBin, mysql.UnsignedFlag | mysql.BinaryFlag, 10, 0}, - {"ceil(c_double_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 22, 0}, - {"floor(c_udouble_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 22, 0}, - {"ceil(c_float_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 12, 0}, - {"floor(c_ufloat_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 12, 0}, - {"ceil(c_datetime)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"ceil(c_timestamp_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"ceil(c_time_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"ceil(c_enum)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"ceil(c_text_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"ceil(18446744073709551615)", mysql.TypeLonglong, charset.CharsetBin, mysql.NotNullFlag | mysql.UnsignedFlag | mysql.BinaryFlag, 20, 0}, + {"ceil(c_int_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"ceil(c_uint_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.UnsignedFlag | mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"ceil(c_bigint_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"ceil(c_ubigint_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.UnsignedFlag | mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"ceil(c_decimal)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"floor(c_udecimal)", mysql.TypeLonglong, charset.CharsetBin, mysql.UnsignedFlag | mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"ceil(c_double_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"floor(c_udouble_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"ceil(c_float_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"floor(c_ufloat_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"ceil(c_datetime)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"ceil(c_timestamp_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"ceil(c_time_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"ceil(c_enum)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"ceil(c_text_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"ceil(18446744073709551615)", mysql.TypeLonglong, charset.CharsetBin, mysql.NotNullFlag | mysql.UnsignedFlag | mysql.BinaryFlag, mysql.MaxIntWidth, 0}, {"ceil(18446744073709551615.1)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 23, 0}, - {"ceiling(c_int_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 11, 0}, - {"ceiling(c_decimal)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 6, 0}, - {"ceiling(c_double_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 22, 0}, - {"ceiling(c_float_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 12, 0}, - {"ceiling(c_datetime)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"ceiling(c_time_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"ceiling(c_enum)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"ceiling(c_text_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"ceiling(18446744073709551615)", mysql.TypeLonglong, charset.CharsetBin, mysql.NotNullFlag | mysql.UnsignedFlag | mysql.BinaryFlag, 20, 0}, + {"ceiling(c_int_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"ceiling(c_decimal)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"ceiling(c_double_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"ceiling(c_float_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"ceiling(c_datetime)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"ceiling(c_time_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"ceiling(c_enum)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"ceiling(c_text_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"ceiling(18446744073709551615)", mysql.TypeLonglong, charset.CharsetBin, mysql.NotNullFlag | mysql.UnsignedFlag | mysql.BinaryFlag, mysql.MaxIntWidth, 0}, {"ceiling(18446744073709551615.1)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 23, 0}, {"conv(c_char, c_int_d, c_int_d)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 64, types.UnspecifiedLength}, @@ -614,29 +614,29 @@ func (s *InferTypeSuite) createTestCase4MathFuncs() []typeInferTestCase { {"abs(c_set )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 22, types.UnspecifiedLength}, {"abs(c_enum )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 22, types.UnspecifiedLength}, - {"round(c_int_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 11, 0}, - {"round(c_bigint_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 20, 0}, - {"round(c_float_d )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 12, 0}, // flen Should be 17. - {"round(c_double_d )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 22, 0}, // flen Should be 17. + {"round(c_int_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"round(c_bigint_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"round(c_float_d )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"round(c_double_d )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, {"round(c_decimal )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 6, 0}, // flen Should be 5. - {"round(c_datetime )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"round(c_time_d )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"round(c_timestamp_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"round(c_char )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"round(c_varchar )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"round(c_text_d )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"round(c_binary )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"round(c_varbinary )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"round(c_blob_d )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"round(c_set )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - {"round(c_enum )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0}, - - {"truncate(c_int_d, 1)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 11, 0}, - {"truncate(c_int_d, -5)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 11, 0}, - {"truncate(c_int_d, 100)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 11, 0}, - {"truncate(c_double_d, 1)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 24, 1}, - {"truncate(c_double_d, 5)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 28, 5}, - {"truncate(c_double_d, 100)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 53, 30}, + {"round(c_datetime )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"round(c_time_d )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"round(c_timestamp_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"round(c_char )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"round(c_varchar )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"round(c_text_d )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"round(c_binary )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"round(c_varbinary )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"round(c_blob_d )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"round(c_set )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"round(c_enum )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + + {"truncate(c_int_d, 1)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"truncate(c_int_d, -5)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"truncate(c_int_d, 100)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0}, + {"truncate(c_double_d, 1)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"truncate(c_double_d, 5)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, + {"truncate(c_double_d, 100)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, {"rand( )", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, {"rand(c_int_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength}, diff --git a/go.mod b/go.mod index 165ed4ad2f538..4f83f251d4d54 100644 --- a/go.mod +++ b/go.mod @@ -65,7 +65,7 @@ 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.20211115071040-a3f1c41ac1a0 + github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211118154139-b11da6307c6f 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 @@ -89,7 +89,7 @@ require ( google.golang.org/grpc v1.40.0 gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect gopkg.in/yaml.v2 v2.4.0 - modernc.org/mathutil v1.2.2 + modernc.org/mathutil v1.4.1 sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) diff --git a/go.sum b/go.sum index 804792f146573..4e8fbc01390bd 100644 --- a/go.sum +++ b/go.sum @@ -160,14 +160,11 @@ github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsr github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/creack/pty v1.1.11/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= -github.com/cznic/golex v0.0.0-20181122101858-9c343928389c/go.mod h1:+bmmJDNmKlhWNG+gwWCkaBoTy39Fs+bzRxVBzoTQbIc= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= -github.com/cznic/parser v0.0.0-20160622100904-31edd927e5b1/go.mod h1:2B43mz36vGZNZEwkWi8ayRSSUXLfjL8OkbzwW4NcPMM= github.com/cznic/sortutil v0.0.0-20181122101858-f5f958428db8 h1:LpMLYGyy67BoAFGda1NeOBQwqlv7nUXpm+rIVHGxZZ4= github.com/cznic/sortutil v0.0.0-20181122101858-f5f958428db8/go.mod h1:q2w6Bg5jeox1B+QkJ6Wp/+Vn0G/bo3f1uY7Fn3vivIQ= github.com/cznic/strutil v0.0.0-20171016134553-529a34b1c186/go.mod h1:AHHPPPXTw0h6pVabbcbyGRK1DckRn7r/STdZEeIDzZc= -github.com/cznic/y v0.0.0-20170802143616-045f81c6662a/go.mod h1:1rk5VM7oSnA4vjp+hrLQ3HWHa+Y4yPCa3/CsJrcNnvs= github.com/danjacques/gofslock v0.0.0-20191023191349-0a45f885bc37 h1:X6mKGhCFOxrKeeHAjv/3UvT6e5RRxW6wRdlqlV6/H4w= github.com/danjacques/gofslock v0.0.0-20191023191349-0a45f885bc37/go.mod h1:DC3JtzuG7kxMvJ6dZmf2ymjNyoXwgtklr7FN+Um2B0U= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -713,8 +710,8 @@ 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.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/client-go/v2 v2.0.0-alpha.0.20211118154139-b11da6307c6f h1:UyJjp3wGIjf1edGiQiIdAtL5QFqaqR4+s3LDwUZU7NY= +github.com/tikv/client-go/v2 v2.0.0-alpha.0.20211118154139-b11da6307c6f/go.mod h1:BEAS0vXm5BorlF/HTndqGwcGDvaiwe7B7BkfgwwZMJ4= 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= @@ -842,6 +839,7 @@ golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a h1:vclmkQCjlDX5OydZ9wv8rB golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20181106170214-d68db9428509/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190125153040-c74c464bbbf2/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1281,8 +1279,20 @@ honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9 honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= honnef.co/go/tools v0.2.0/go.mod h1:lPVVZ2BS5TfnjLyizF7o7hv7j9/L+8cZY2hLyjP9cGY= k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= -modernc.org/mathutil v1.2.2 h1:+yFk8hBprV+4c0U9GjFtL+dV3N8hOJ8JCituQcMShFY= -modernc.org/mathutil v1.2.2/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= +modernc.org/fileutil v1.0.0/go.mod h1:JHsWpkrk/CnVV1H/eGlFf85BEpfkrp56ro8nojIq9Q8= +modernc.org/golex v1.0.1/go.mod h1:QCA53QtsT1NdGkaZZkF5ezFwk4IXh4BGNafAARTC254= +modernc.org/lex v1.0.0/go.mod h1:G6rxMTy3cH2iA0iXL/HRRv4Znu8MK4higxph/lE7ypk= +modernc.org/lexer v1.0.0/go.mod h1:F/Dld0YKYdZCLQ7bD0USbWL4YKCyTDRDHiDTOs0q0vk= +modernc.org/mathutil v1.0.0/go.mod h1:wU0vUrJsVWBZ4P6e7xtFJEhFSNsfRLJ8H458uRjg03k= +modernc.org/mathutil v1.4.1 h1:ij3fYGe8zBF4Vu+g0oT7mB06r8sqGWKuJu1yXeR4by8= +modernc.org/mathutil v1.4.1/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= +modernc.org/parser v1.0.0/go.mod h1:H20AntYJ2cHHL6MHthJ8LZzXCdDCHMWt1KZXtIMjejA= +modernc.org/parser v1.0.2/go.mod h1:TXNq3HABP3HMaqLK7brD1fLA/LfN0KS6JxZn71QdDqs= +modernc.org/scanner v1.0.1/go.mod h1:OIzD2ZtjYk6yTuyqZr57FmifbM9fIH74SumloSsajuE= +modernc.org/sortutil v1.0.0/go.mod h1:1QO0q8IlIlmjBIwm6t/7sof874+xCfZouyqZMLIAtxM= +modernc.org/strutil v1.0.0/go.mod h1:lstksw84oURvj9y3tn8lGvRxyRC1S2+g5uuIzNfIOBs= +modernc.org/strutil v1.1.0/go.mod h1:lstksw84oURvj9y3tn8lGvRxyRC1S2+g5uuIzNfIOBs= +modernc.org/y v1.0.1/go.mod h1:Ho86I+LVHEI+LYXoUKlmOMAM1JTXOCfj8qi1T8PsClE= moul.io/zapgorm2 v1.1.0/go.mod h1:emRfKjNqSzVj5lcgasBdovIXY1jSOwFz2GQZn1Rddks= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= diff --git a/infoschema/builder.go b/infoschema/builder.go index d3c532a2bfffb..d0ab2d58ac273 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -41,6 +41,8 @@ type Builder struct { // TODO: store is only used by autoid allocators // detach allocators from storage, use passed transaction in the feature store kv.Storage + // TODO: renewLeaseCh is only used to pass data between table and domain + renewLeaseCh chan func() } // ApplyDiff applies SchemaDiff to the new InfoSchema. @@ -438,7 +440,7 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i } } } - tbl, err := tables.TableFromMeta(allocs, tblInfo) + tbl, err := b.tableFromMeta(allocs, tblInfo) if err != nil { return nil, errors.Trace(err) } @@ -601,7 +603,7 @@ func (b *Builder) InitWithDBInfos(dbInfos []*model.DBInfo, bundles []*placement. } for _, di := range dbInfos { - err := b.createSchemaTablesForDB(di, tables.TableFromMeta) + err := b.createSchemaTablesForDB(di, b.tableFromMeta) if err != nil { return nil, errors.Trace(err) } @@ -622,6 +624,20 @@ func (b *Builder) InitWithDBInfos(dbInfos []*model.DBInfo, bundles []*placement. return b, nil } +func (b *Builder) tableFromMeta(alloc autoid.Allocators, tblInfo *model.TableInfo) (table.Table, error) { + ret, err := tables.TableFromMeta(alloc, tblInfo) + if err != nil { + return nil, errors.Trace(err) + } + if t, ok := ret.(table.CachedTable); ok { + err = t.Init(b.renewLeaseCh) + if err != nil { + return nil, errors.Trace(err) + } + } + return ret, nil +} + type tableFromMetaFunc func(alloc autoid.Allocators, tblInfo *model.TableInfo) (table.Table, error) func (b *Builder) createSchemaTablesForDB(di *model.DBInfo, tableFromMeta tableFromMetaFunc) error { @@ -658,7 +674,7 @@ func RegisterVirtualTable(dbInfo *model.DBInfo, tableFromMeta tableFromMetaFunc) } // NewBuilder creates a new Builder with a Handle. -func NewBuilder(store kv.Storage) *Builder { +func NewBuilder(store kv.Storage, renewCh chan func()) *Builder { return &Builder{ store: store, is: &infoSchema{ @@ -667,6 +683,7 @@ func NewBuilder(store kv.Storage) *Builder { ruleBundleMap: map[string]*placement.Bundle{}, sortedTablesBuckets: make([]sortedTables, bucketCount), }, + renewLeaseCh: renewCh, } } diff --git a/infoschema/cluster_tables_serial_test.go b/infoschema/cluster_tables_serial_test.go index b39aa73f2efd6..a9cfd496c4b8d 100644 --- a/infoschema/cluster_tables_serial_test.go +++ b/infoschema/cluster_tables_serial_test.go @@ -43,6 +43,7 @@ import ( "github.com/pingcap/tidb/util/resourcegrouptag" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/testutil" + "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" "google.golang.org/grpc" ) @@ -160,10 +161,10 @@ func SubTestTestDataLockWaits(s *clusterTablesSuite) func(*testing.T) { _, digest1 := parser.NormalizeDigest("select * from test_data_lock_waits for update") _, digest2 := parser.NormalizeDigest("update test_data_lock_waits set f1=1 where id=2") s.store.(mockstorage.MockLockWaitSetter).SetMockLockWaits([]*deadlock.WaitForEntry{ - {Txn: 1, WaitForTxn: 2, Key: []byte("key1"), ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(digest1, nil)}, - {Txn: 3, WaitForTxn: 4, Key: []byte("key2"), ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(digest2, nil)}, + {Txn: 1, WaitForTxn: 2, Key: []byte("key1"), ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(digest1, nil, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown)}, + {Txn: 3, WaitForTxn: 4, Key: []byte("key2"), ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(digest2, nil, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown)}, // Invalid digests - {Txn: 5, WaitForTxn: 6, Key: []byte("key3"), ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(nil, nil)}, + {Txn: 5, WaitForTxn: 6, Key: []byte("key3"), ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(nil, nil, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown)}, {Txn: 7, WaitForTxn: 8, Key: []byte("key4"), ResourceGroupTag: []byte("asdfghjkl")}, }) diff --git a/infoschema/infoschema_test.go b/infoschema/infoschema_test.go index 1694c481d24dd..87c1695833f4d 100644 --- a/infoschema/infoschema_test.go +++ b/infoschema/infoschema_test.go @@ -109,7 +109,7 @@ func TestBasic(t *testing.T) { }) require.NoError(t, err) - builder, err := infoschema.NewBuilder(dom.Store()).InitWithDBInfos(dbInfos, nil, nil, 1) + builder, err := infoschema.NewBuilder(dom.Store(), nil).InitWithDBInfos(dbInfos, nil, nil, 1) require.NoError(t, err) txn, err := store.Begin() @@ -259,7 +259,7 @@ func TestInfoTables(t *testing.T) { require.NoError(t, err) }() - builder, err := infoschema.NewBuilder(store).InitWithDBInfos(nil, nil, nil, 0) + builder, err := infoschema.NewBuilder(store, nil).InitWithDBInfos(nil, nil, nil, 0) require.NoError(t, err) is := builder.Build() @@ -326,7 +326,7 @@ func TestGetBundle(t *testing.T) { require.NoError(t, err) }() - builder, err := infoschema.NewBuilder(store).InitWithDBInfos(nil, nil, nil, 0) + builder, err := infoschema.NewBuilder(store, nil).InitWithDBInfos(nil, nil, nil, 0) require.NoError(t, err) is := builder.Build() diff --git a/kv/kv.go b/kv/kv.go index a6ce83fd8caf0..22610d17b8d9a 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -30,6 +30,7 @@ import ( tikvstore "github.com/tikv/client-go/v2/kv" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikv" + "github.com/tikv/client-go/v2/tikvrpc" ) // UnCommitIndexKVFlag uses to indicate the index key/value is no need to commit. @@ -335,8 +336,8 @@ type Request struct { IsStaleness bool // MatchStoreLabels indicates the labels the store should be matched MatchStoreLabels []*metapb.StoreLabel - // ResourceGroupTag indicates the kv request task group. - ResourceGroupTag []byte + // ResourceGroupTagger indicates the kv request task group tagger. + ResourceGroupTagger tikvrpc.ResourceGroupTagger } const ( diff --git a/kv/option.go b/kv/option.go index 682c2be4f2d60..683f5fcb8e389 100644 --- a/kv/option.go +++ b/kv/option.go @@ -62,10 +62,13 @@ const ( IsStalenessReadOnly // MatchStoreLabels indicates the labels the store should be matched MatchStoreLabels - // ResourceGroupTag indicates the resource group of the kv request. + // ResourceGroupTag indicates the resource group tag of the kv request. ResourceGroupTag + // ResourceGroupTagger can be used to set the ResourceGroupTag dynamically according to the request content. It will be used only when ResourceGroupTag is nil. + ResourceGroupTagger // KVFilter indicates the filter to ignore key-values in the transaction's memory buffer. KVFilter + // SnapInterceptor is used for setting the interceptor for snapshot SnapInterceptor ) diff --git a/lock/lock.go b/lock/lock.go index b613ff1018b11..71d76ed924075 100644 --- a/lock/lock.go +++ b/lock/lock.go @@ -15,6 +15,8 @@ package lock import ( + "errors" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -29,6 +31,9 @@ type Checker struct { is infoschema.InfoSchema } +// ErrLockedTableDropped returns error when try to drop the table with write lock +var ErrLockedTableDropped = errors.New("other table can be accessed after locked table dropped") + // NewChecker return new lock Checker. func NewChecker(ctx sessionctx.Context, is infoschema.InfoSchema) *Checker { return &Checker{ctx: ctx, is: is} @@ -47,6 +52,7 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType if !alterWriteable && table == "" { return c.CheckLockInDB(db, privilege) } + switch privilege { case mysql.ShowDBPriv, mysql.AllPrivMask: // AllPrivMask only used in show create table statement now. @@ -68,6 +74,24 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType if err != nil { return err } + if tb.Meta().Lock == nil { + return nil + } + if privilege == mysql.DropPriv && tb.Meta().Name.O == table && c.ctx.HasLockedTables() { + lockTables := c.ctx.GetAllTableLocks() + for _, lockT := range lockTables { + if lockT.TableID == tb.Meta().ID { + switch tb.Meta().Lock.Tp { + case model.TableLockWrite: + return ErrLockedTableDropped + case model.TableLockRead, model.TableLockWriteLocal, model.TableLockReadOnly: + return infoschema.ErrTableNotLockedForWrite.GenWithStackByArgs(tb.Meta().Name) + } + } + } + + } + if !alterWriteable && c.ctx.HasLockedTables() { if locked, tp := c.ctx.CheckTableLocked(tb.Meta().ID); locked { if checkLockTpMeetPrivilege(tp, privilege) { @@ -78,10 +102,6 @@ func (c *Checker) CheckTableLock(db, table string, privilege mysql.PrivilegeType return infoschema.ErrTableNotLocked.GenWithStackByArgs(tb.Meta().Name) } - if tb.Meta().Lock == nil { - return nil - } - if privilege == mysql.SelectPriv { switch tb.Meta().Lock.Tp { case model.TableLockRead, model.TableLockWriteLocal, model.TableLockReadOnly: diff --git a/metrics/grafana/overview.json b/metrics/grafana/overview.json index 61bcb3543640b..ebbfb72ea361c 100644 --- a/metrics/grafana/overview.json +++ b/metrics/grafana/overview.json @@ -2695,6 +2695,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "The duration of a client starting to wait for the TS until received the TS result.", "fill": 1, "gridPos": { "h": 7, @@ -2730,7 +2731,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", + "expr": "histogram_quantile(0.999, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "999", @@ -2738,14 +2739,14 @@ "step": 10 }, { - "expr": "histogram_quantile(0.99, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", "refId": "B" }, { - "expr": "histogram_quantile(0.90, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", + "expr": "histogram_quantile(0.90, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "90", diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index cddda9e6063e7..0a12c363b0ea1 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -8521,7 +8521,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The duration of the waiting time for getting the start timestamp oracle", + "description": "The duration of the async TS until called the Wait function.", "editable": true, "error": false, "fill": 1, @@ -8556,7 +8556,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tidb_pdclient_ts_future_wait_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.999, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso_async_wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "999", @@ -8564,14 +8564,14 @@ "step": 10 }, { - "expr": "histogram_quantile(0.99, sum(rate(tidb_pdclient_ts_future_wait_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso_async_wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", "refId": "B" }, { - "expr": "histogram_quantile(0.90, sum(rate(tidb_pdclient_ts_future_wait_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.90, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso_async_wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "90", @@ -8582,7 +8582,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Start TSO Wait Duration", + "title": "Async TSO Duration", "tooltip": { "msResolution": false, "shared": true, diff --git a/parser/ast/ddl.go b/parser/ast/ddl.go index 917fadb3ed566..b537012a9c226 100644 --- a/parser/ast/ddl.go +++ b/parser/ast/ddl.go @@ -1435,6 +1435,7 @@ func (n *CreateViewStmt) Accept(v Visitor) (Node, bool) { type CreatePlacementPolicyStmt struct { ddlNode + OrReplace bool IfNotExists bool PolicyName model.CIStr PlacementOptions []*PlacementOption @@ -1442,7 +1443,11 @@ type CreatePlacementPolicyStmt struct { // Restore implements Node interface. func (n *CreatePlacementPolicyStmt) Restore(ctx *format.RestoreCtx) error { - ctx.WriteKeyWord("CREATE PLACEMENT POLICY ") + ctx.WriteKeyWord("CREATE ") + if n.OrReplace { + ctx.WriteKeyWord("OR REPLACE ") + } + ctx.WriteKeyWord("PLACEMENT POLICY ") if n.IfNotExists { ctx.WriteKeyWord("IF NOT EXISTS ") } diff --git a/parser/ast/dml.go b/parser/ast/dml.go index 2349602da70ed..0a2c934f7a597 100644 --- a/parser/ast/dml.go +++ b/parser/ast/dml.go @@ -1100,6 +1100,8 @@ type SelectStmt struct { // Lists is filled only when Kind == SelectStmtKindValues Lists []*RowExpr With *WithClause + // AsViewSchema indicates if this stmt provides the schema for the view. It is only used when creating the view + AsViewSchema bool } func (*SelectStmt) resultSet() {} diff --git a/parser/ast/misc.go b/parser/ast/misc.go index a1852231fb3ff..3e185319e0b11 100644 --- a/parser/ast/misc.go +++ b/parser/ast/misc.go @@ -120,14 +120,17 @@ func (n *AuthOption) Restore(ctx *format.RestoreCtx) error { type TraceStmt struct { stmtNode - Stmt StmtNode - Format string + Stmt StmtNode + Format string + TracePlan bool } // Restore implements Node interface. func (n *TraceStmt) Restore(ctx *format.RestoreCtx) error { ctx.WriteKeyWord("TRACE ") - if n.Format != "row" { + if n.TracePlan { + ctx.WriteKeyWord("PLAN ") + } else if n.Format != "row" { ctx.WriteKeyWord("FORMAT") ctx.WritePlain(" = ") ctx.WriteString(n.Format) diff --git a/parser/charset/encoding.go b/parser/charset/encoding.go index 72bff5b6cf7e9..8bd1b92c9bcf6 100644 --- a/parser/charset/encoding.go +++ b/parser/charset/encoding.go @@ -28,8 +28,6 @@ import ( "golang.org/x/text/transform" ) -const encodingLegacy = "utf-8" // utf-8 encoding is compatible with old default behavior. - var errInvalidCharacterString = terror.ClassParser.NewStd(mysql.ErrInvalidCharacterString) type EncodingLabel string @@ -54,7 +52,7 @@ type Encoding struct { // enabled indicates whether the non-utf8 encoding is used. func (e *Encoding) enabled() bool { - return e.enc != nil && e.charLength != nil + return e != UTF8Encoding } // Name returns the name of the current encoding. @@ -62,35 +60,21 @@ func (e *Encoding) Name() string { return e.name } +// CharLength returns the next character length in bytes. +func (e *Encoding) CharLength(bs []byte) int { + return e.charLength(bs) +} + // NewEncoding creates a new Encoding. func NewEncoding(label string) *Encoding { if len(label) == 0 { - return &Encoding{} + return UTF8Encoding } - e, name := Lookup(label) - if e != nil && name != encodingLegacy { - return &Encoding{ - enc: e, - name: name, - charLength: FindNextCharacterLength(name), - specialCase: LookupSpecialCase(name), - } - } - return &Encoding{name: name} -} - -// UpdateEncoding updates to a new Encoding. -func (e *Encoding) UpdateEncoding(label EncodingLabel) { - enc, name := lookup(label) - e.name = name - if enc != nil && name != encodingLegacy { - e.enc = enc - e.charLength = FindNextCharacterLength(name) - } else { - e.enc = nil - e.charLength = nil + + if e, exist := encodingMap[Format(label)]; exist { + return e } - e.specialCase = LookupSpecialCase(e.name) + return UTF8Encoding } // Encode convert bytes from utf-8 charset to a specific charset. @@ -110,6 +94,16 @@ func (e *Encoding) EncodeString(src string) (string, error) { return string(bs), err } +// EncodeFirstChar convert first code point of bytes from utf-8 charset to a specific charset. +func (e *Encoding) EncodeFirstChar(dest, src []byte) ([]byte, error) { + srcNextLen := e.nextCharLenInSrc(src, false) + srcEnd := mathutil.Min(srcNextLen, len(src)) + if !e.enabled() { + return src[:srcEnd], nil + } + return e.transform(e.enc.NewEncoder(), dest, src[:srcEnd], false) +} + // EncodeInternal convert bytes from utf-8 charset to a specific charset, we actually do not do the real convert, just find the inconvertible character and use ? replace. // The code below is equivalent to // expr, _ := e.Encode(dest, src) @@ -127,7 +121,7 @@ func (e *Encoding) EncodeInternal(dest, src []byte) []byte { var buf [4]byte transformer := e.enc.NewEncoder() for srcOffset < len(src) { - length := characterLengthUTF8(src[srcOffset:]) + length := UTF8Encoding.CharLength(src[srcOffset:]) _, _, err := transformer.Transform(buf[:], src[srcOffset:srcOffset+length], true) if err != nil { dest = append(dest, byte('?')) @@ -161,6 +155,9 @@ func (e *Encoding) transform(transformer transform.Transformer, dest, src []byte if len(dest) < len(src) { dest = make([]byte, len(src)*2) } + if len(src) == 0 { + return src, nil + } var destOffset, srcOffset int var encodingErr error for { @@ -192,7 +189,7 @@ func (e *Encoding) nextCharLenInSrc(srcRest []byte, isDecoding bool) int { } return len(srcRest) } - return characterLengthUTF8(srcRest) + return UTF8Encoding.CharLength(srcRest) } func enlargeCapacity(dest []byte) []byte { diff --git a/parser/charset/encoding_table.go b/parser/charset/encoding_table.go index ea7e6d8915798..2de9d957d923a 100644 --- a/parser/charset/encoding_table.go +++ b/parser/charset/encoding_table.go @@ -15,7 +15,10 @@ package charset import ( "strings" + go_unicode "unicode" + "unicode/utf8" + "github.com/cznic/mathutil" "golang.org/x/text/encoding" "golang.org/x/text/encoding/charmap" "golang.org/x/text/encoding/japanese" @@ -25,6 +28,15 @@ import ( "golang.org/x/text/encoding/unicode" ) +var encodingMap = map[EncodingLabel]*Encoding{ + CharsetUTF8MB4: UTF8Encoding, + CharsetUTF8: UTF8Encoding, + CharsetGBK: GBKEncoding, + CharsetLatin1: LatinEncoding, + CharsetBin: BinaryEncoding, + CharsetASCII: ASCIIEncoding, +} + // Lookup returns the encoding with the specified label, and its canonical // name. It returns nil and the empty string if label is not one of the // standard encodings for HTML. Matching is case-insensitive and ignores @@ -263,38 +275,178 @@ var encodings = map[string]struct { "x-user-defined": {charmap.XUserDefined, "x-user-defined"}, } -// FindNextCharacterLength is used in lexer.peek() to determine the next character length. -func FindNextCharacterLength(label string) func([]byte) int { - if f, ok := encodingNextCharacterLength[label]; ok { - return f +// TruncateStrategy indicates the way to handle the invalid strings in specific charset. +// - TruncateStrategyEmpty: returns an empty string. +// - TruncateStrategyTrim: returns the valid prefix part of string. +// - TruncateStrategyReplace: returns the whole string, but the invalid characters are replaced with '?'. +type TruncateStrategy int8 + +const ( + TruncateStrategyEmpty TruncateStrategy = iota + TruncateStrategyTrim + TruncateStrategyReplace +) + +var _ StringValidator = StringValidatorASCII{} +var _ StringValidator = StringValidatorUTF8{} +var _ StringValidator = StringValidatorOther{} + +// StringValidator is used to check if a string is valid in the specific charset. +type StringValidator interface { + Validate(str string) (invalidPos int) + Truncate(str string, strategy TruncateStrategy) (result string, invalidPos int) +} + +// StringValidatorASCII checks whether a string is valid ASCII string. +type StringValidatorASCII struct{} + +// Validate checks whether the string is valid in the given charset. +func (s StringValidatorASCII) Validate(str string) int { + _, invalidPos := s.Truncate(str, TruncateStrategyEmpty) + return invalidPos +} + +// Truncate implement the interface StringValidator. +func (s StringValidatorASCII) Truncate(str string, strategy TruncateStrategy) (string, int) { + invalidPos := -1 + for i := 0; i < len(str); i++ { + if str[i] > go_unicode.MaxASCII { + invalidPos = i + break + } } - return nil + if invalidPos == -1 { + // Quick check passed. + return str, -1 + } + switch strategy { + case TruncateStrategyEmpty: + return "", invalidPos + case TruncateStrategyTrim: + return str[:invalidPos], invalidPos + case TruncateStrategyReplace: + result := make([]byte, 0, len(str)) + for i, w := 0, 0; i < len(str); i += w { + w = 1 + if str[i] > go_unicode.MaxASCII { + w = UTF8Encoding.CharLength(Slice(str)[i:]) + w = mathutil.Min(w, len(str)-i) + result = append(result, '?') + continue + } + result = append(result, str[i:i+w]...) + } + return string(result), invalidPos + } + return str, -1 } -var encodingNextCharacterLength = map[string]func([]byte) int{ - // https://en.wikipedia.org/wiki/GBK_(character_encoding)#Layout_diagram - "gbk": characterLengthGBK, - "utf-8": characterLengthUTF8, - "binary": func(bs []byte) int { - return 1 - }, +// StringValidatorUTF8 checks whether a string is valid UTF8 string. +type StringValidatorUTF8 struct { + IsUTF8MB4 bool // Distinguish between "utf8" and "utf8mb4" + CheckMB4ValueInUTF8 bool } -func characterLengthGBK(bs []byte) int { - if len(bs) == 0 || bs[0] < 0x80 { - // A byte in the range 00–7F is a single byte that means the same thing as it does in ASCII. - return 1 +// Validate checks whether the string is valid in the given charset. +func (s StringValidatorUTF8) Validate(str string) int { + _, invalidPos := s.Truncate(str, TruncateStrategyEmpty) + return invalidPos +} + +// Truncate implement the interface StringValidator. +func (s StringValidatorUTF8) Truncate(str string, strategy TruncateStrategy) (string, int) { + if str == "" { + return str, -1 + } + if s.IsUTF8MB4 && utf8.ValidString(str) { + // Quick check passed. + return str, -1 + } + doMB4CharCheck := !s.IsUTF8MB4 && s.CheckMB4ValueInUTF8 + var result []byte + if strategy == TruncateStrategyReplace { + result = make([]byte, 0, len(str)) + } + invalidPos := -1 + for i, w := 0, 0; i < len(str); i += w { + var rv rune + rv, w = utf8.DecodeRuneInString(str[i:]) + if (rv == utf8.RuneError && w == 1) || (w > 3 && doMB4CharCheck) { + if invalidPos == -1 { + invalidPos = i + } + switch strategy { + case TruncateStrategyEmpty: + return "", invalidPos + case TruncateStrategyTrim: + return str[:i], invalidPos + case TruncateStrategyReplace: + result = append(result, '?') + continue + } + } + if strategy == TruncateStrategyReplace { + result = append(result, str[i:i+w]...) + } } - return 2 + if strategy == TruncateStrategyReplace { + return string(result), invalidPos + } + return str, -1 +} + +// StringValidatorOther checks whether a string is valid string in given charset. +type StringValidatorOther struct { + Charset string } -func characterLengthUTF8(bs []byte) int { - if len(bs) == 0 || bs[0] < 0x80 { - return 1 - } else if bs[0] < 0xe0 { - return 2 - } else if bs[0] < 0xf0 { - return 3 +// Validate checks whether the string is valid in the given charset. +func (s StringValidatorOther) Validate(str string) int { + _, invalidPos := s.Truncate(str, TruncateStrategyEmpty) + return invalidPos +} + +// Truncate implement the interface StringValidator. +func (s StringValidatorOther) Truncate(str string, strategy TruncateStrategy) (string, int) { + if str == "" { + return str, -1 + } + enc := NewEncoding(s.Charset) + if !enc.enabled() { + return str, -1 + } + var result []byte + if strategy == TruncateStrategyReplace { + result = make([]byte, 0, len(str)) + } + var buf [4]byte + strBytes := Slice(str) + transformer := enc.enc.NewEncoder() + invalidPos := -1 + for i, w := 0, 0; i < len(str); i += w { + w = UTF8Encoding.CharLength(strBytes[i:]) + w = mathutil.Min(w, len(str)-i) + _, _, err := transformer.Transform(buf[:], strBytes[i:i+w], true) + if err != nil { + if invalidPos == -1 { + invalidPos = i + } + switch strategy { + case TruncateStrategyEmpty: + return "", invalidPos + case TruncateStrategyTrim: + return str[:i], invalidPos + case TruncateStrategyReplace: + result = append(result, '?') + continue + } + } + if strategy == TruncateStrategyReplace { + result = append(result, strBytes[i:i+w]...) + } + } + if strategy == TruncateStrategyReplace { + return string(result), invalidPos } - return 4 + return str, -1 } diff --git a/parser/charset/encoding_test.go b/parser/charset/encoding_test.go index fd6a4d062c467..32ae8496c2e93 100644 --- a/parser/charset/encoding_test.go +++ b/parser/charset/encoding_test.go @@ -16,6 +16,7 @@ package charset_test import ( "fmt" "testing" + "unicode/utf8" "github.com/pingcap/tidb/parser/charset" "github.com/stretchr/testify/require" @@ -24,12 +25,8 @@ import ( func TestEncoding(t *testing.T) { t.Parallel() - enc := charset.NewEncoding("gbk") - require.Equal(t, "gbk", enc.Name()) - enc.UpdateEncoding("utf-8") - require.Equal(t, "utf-8", enc.Name()) - enc.UpdateEncoding("gbk") - require.Equal(t, "gbk", enc.Name()) + enc := charset.NewEncoding(charset.CharsetGBK) + require.Equal(t, charset.CharsetGBK, enc.Name()) txt := []byte("一二三四") e, _ := charset.Lookup("gbk") @@ -91,3 +88,112 @@ func TestEncoding(t *testing.T) { require.Equal(t, tc.result, string(result), cmt) } } + +func TestStringValidatorASCII(t *testing.T) { + v := charset.StringValidatorASCII{} + testCases := []struct { + str string + strategy charset.TruncateStrategy + expected string + invalidPos int + }{ + {"", charset.TruncateStrategyEmpty, "", -1}, + {"qwerty", charset.TruncateStrategyEmpty, "qwerty", -1}, + {"qwÊrty", charset.TruncateStrategyEmpty, "", 2}, + {"qwÊrty", charset.TruncateStrategyTrim, "qw", 2}, + {"qwÊrty", charset.TruncateStrategyReplace, "qw?rty", 2}, + {"中文", charset.TruncateStrategyEmpty, "", 0}, + {"中文?qwert", charset.TruncateStrategyTrim, "", 0}, + {"中文?qwert", charset.TruncateStrategyReplace, "???qwert", 0}, + } + for _, tc := range testCases { + msg := fmt.Sprintf("%v", tc) + actual, invalidPos := v.Truncate(tc.str, tc.strategy) + require.Equal(t, tc.expected, actual, msg) + require.Equal(t, tc.invalidPos, invalidPos, msg) + } + require.Equal(t, -1, v.Validate("qwerty")) + require.Equal(t, 2, v.Validate("qwÊrty")) + require.Equal(t, 0, v.Validate("中文")) +} + +func TestStringValidatorUTF8(t *testing.T) { + // Test charset "utf8mb4". + v := charset.StringValidatorUTF8{IsUTF8MB4: true} + oxfffefd := string([]byte{0xff, 0xfe, 0xfd}) + testCases := []struct { + str string + strategy charset.TruncateStrategy + expected string + invalidPos int + }{ + {"", charset.TruncateStrategyEmpty, "", -1}, + {"qwerty", charset.TruncateStrategyEmpty, "qwerty", -1}, + {"qwÊrty", charset.TruncateStrategyEmpty, "qwÊrty", -1}, + {"qwÊ合法字符串", charset.TruncateStrategyEmpty, "qwÊ合法字符串", -1}, + {"😂", charset.TruncateStrategyEmpty, "😂", -1}, + {oxfffefd, charset.TruncateStrategyEmpty, "", 0}, + {oxfffefd, charset.TruncateStrategyReplace, "???", 0}, + {"中文" + oxfffefd, charset.TruncateStrategyTrim, "中文", 6}, + {"中文" + oxfffefd, charset.TruncateStrategyReplace, "中文???", 6}, + {string(utf8.RuneError), charset.TruncateStrategyEmpty, "�", -1}, + } + for _, tc := range testCases { + msg := fmt.Sprintf("%v", tc) + actual, invalidPos := v.Truncate(tc.str, tc.strategy) + require.Equal(t, tc.expected, actual, msg) + require.Equal(t, tc.invalidPos, invalidPos, msg) + } + // Test charset "utf8" with checking mb4 value. + v = charset.StringValidatorUTF8{IsUTF8MB4: false, CheckMB4ValueInUTF8: true} + testCases = []struct { + str string + strategy charset.TruncateStrategy + expected string + invalidPos int + }{ + {"", charset.TruncateStrategyEmpty, "", -1}, + {"qwerty", charset.TruncateStrategyEmpty, "qwerty", -1}, + {"qwÊrty", charset.TruncateStrategyEmpty, "qwÊrty", -1}, + {"qwÊ合法字符串", charset.TruncateStrategyEmpty, "qwÊ合法字符串", -1}, + {"😂", charset.TruncateStrategyEmpty, "", 0}, + {"😂", charset.TruncateStrategyReplace, "?", 0}, + {"valid_str😂", charset.TruncateStrategyReplace, "valid_str?", 9}, + {oxfffefd, charset.TruncateStrategyEmpty, "", 0}, + {oxfffefd, charset.TruncateStrategyReplace, "???", 0}, + {"中文" + oxfffefd, charset.TruncateStrategyTrim, "中文", 6}, + {"中文" + oxfffefd, charset.TruncateStrategyReplace, "中文???", 6}, + {string(utf8.RuneError), charset.TruncateStrategyEmpty, "�", -1}, + } + for _, tc := range testCases { + msg := fmt.Sprintf("%v", tc) + actual, invalidPos := v.Truncate(tc.str, tc.strategy) + require.Equal(t, tc.expected, actual, msg) + require.Equal(t, tc.invalidPos, invalidPos, msg) + } +} + +func TestStringValidatorGBK(t *testing.T) { + v := charset.StringValidatorOther{Charset: "gbk"} + testCases := []struct { + str string + strategy charset.TruncateStrategy + expected string + invalidPos int + }{ + {"", charset.TruncateStrategyEmpty, "", -1}, + {"asdf", charset.TruncateStrategyEmpty, "asdf", -1}, + {"中文", charset.TruncateStrategyEmpty, "中文", -1}, + {"À", charset.TruncateStrategyEmpty, "", 0}, + {"À", charset.TruncateStrategyReplace, "?", 0}, + {"中文À中文", charset.TruncateStrategyTrim, "中文", 6}, + {"中文À中文", charset.TruncateStrategyReplace, "中文?中文", 6}, + {"asdfÀ", charset.TruncateStrategyReplace, "asdf?", 4}, + } + for _, tc := range testCases { + msg := fmt.Sprintf("%v", tc) + actual, invalidPos := v.Truncate(tc.str, tc.strategy) + require.Equal(t, tc.expected, actual, msg) + require.Equal(t, tc.invalidPos, invalidPos, msg) + } +} diff --git a/parser/charset/gbk.go b/parser/charset/gbk.go new file mode 100644 index 0000000000000..5686c6e1b50f0 --- /dev/null +++ b/parser/charset/gbk.go @@ -0,0 +1,29 @@ +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package charset + +import "golang.org/x/text/encoding/simplifiedchinese" + +var GBKEncoding = &Encoding{ + enc: simplifiedchinese.GBK, + name: CharsetGBK, + charLength: func(bs []byte) int { + if len(bs) == 0 || bs[0] < 0x80 { + // A byte in the range 00–7F is a single byte that means the same thing as it does in ASCII. + return 1 + } + return 2 + }, + specialCase: GBKCase, +} diff --git a/parser/charset/latin.go b/parser/charset/latin.go new file mode 100644 index 0000000000000..04de80d250aef --- /dev/null +++ b/parser/charset/latin.go @@ -0,0 +1,48 @@ +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package charset + +import ( + "golang.org/x/text/encoding" + "golang.org/x/text/encoding/charmap" +) + +var ( + LatinEncoding = &Encoding{ + enc: charmap.Windows1252, + name: CharsetLatin1, + charLength: func(bytes []byte) int { + return 1 + }, + specialCase: nil, + } + + BinaryEncoding = &Encoding{ + enc: encoding.Nop, + name: CharsetBin, + charLength: func(bytes []byte) int { + return 1 + }, + specialCase: nil, + } + + ASCIIEncoding = &Encoding{ + enc: encoding.Nop, + name: CharsetASCII, + charLength: func(bytes []byte) int { + return 1 + }, + specialCase: nil, + } +) diff --git a/parser/charset/utf.go b/parser/charset/utf.go new file mode 100644 index 0000000000000..301aaba49d19a --- /dev/null +++ b/parser/charset/utf.go @@ -0,0 +1,34 @@ +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package charset + +import ( + "golang.org/x/text/encoding" +) + +var UTF8Encoding = &Encoding{ + enc: encoding.Nop, + name: CharsetUTF8MB4, + charLength: func(bs []byte) int { + if len(bs) == 0 || bs[0] < 0x80 { + return 1 + } else if bs[0] < 0xe0 { + return 2 + } else if bs[0] < 0xf0 { + return 3 + } + return 4 + }, + specialCase: nil, +} diff --git a/parser/checkout-pr-branch.sh b/parser/checkout-pr-branch.sh deleted file mode 100755 index e66b4aad82f4c..0000000000000 --- a/parser/checkout-pr-branch.sh +++ /dev/null @@ -1,24 +0,0 @@ -#!/bin/bash - -# This script is used to checkout a Parser PR branch in a forked repo. -if test -z $1; then - echo -e "Usage:\n" - echo -e "\tcheckout-pr-branch.sh [github-username]:[pr-branch]\n" - echo -e "The argument can be copied directly from github PR page." - echo -e "The local branch name would be [github-username]/[pr-branch]." - exit 0; -fi - -username=$(echo $1 | cut -d':' -f1) -branch=$(echo $1 | cut -d':' -f2) -local_branch=$username/$branch -fork="https://github.com/$username/parser" - -exists=`git show-ref refs/heads/$local_branch` -if [ -n "$exists" ]; then - git checkout $local_branch - git pull $fork $branch:$local_branch -else - git fetch $fork $branch:$local_branch - git checkout $local_branch -fi diff --git a/parser/go.mod b/parser/go.mod index 53b784f02c6ee..02b141eed3261 100644 --- a/parser/go.mod +++ b/parser/go.mod @@ -1,20 +1,18 @@ module github.com/pingcap/tidb/parser require ( - github.com/cznic/golex v0.0.0-20181122101858-9c343928389c // indirect github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 - github.com/cznic/parser v0.0.0-20160622100904-31edd927e5b1 github.com/cznic/sortutil v0.0.0-20181122101858-f5f958428db8 github.com/cznic/strutil v0.0.0-20171016134553-529a34b1c186 - github.com/cznic/y v0.0.0-20170802143616-045f81c6662a github.com/go-sql-driver/mysql v1.6.0 github.com/pingcap/errors v0.11.5-0.20210425183316-da1aaba5fb63 github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7 - github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 // indirect github.com/stretchr/testify v1.7.0 go.uber.org/goleak v1.1.10 go.uber.org/zap v1.18.1 golang.org/x/text v0.3.6 + modernc.org/parser v1.0.2 + modernc.org/y v1.0.1 ) go 1.13 diff --git a/parser/go.sum b/parser/go.sum index 53ae9041e4809..3ea7bfd55e941 100644 --- a/parser/go.sum +++ b/parser/go.sum @@ -2,18 +2,12 @@ github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= -github.com/cznic/golex v0.0.0-20181122101858-9c343928389c h1:G8zTsaqyVfIHpgMFcGgdbhHSFhlNc77rAKkhVbQ9kQg= -github.com/cznic/golex v0.0.0-20181122101858-9c343928389c/go.mod h1:+bmmJDNmKlhWNG+gwWCkaBoTy39Fs+bzRxVBzoTQbIc= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM= -github.com/cznic/parser v0.0.0-20160622100904-31edd927e5b1 h1:uWcWCkSP+E1w1z8r082miT+c+9vzg+5UdrgGCo15lMo= -github.com/cznic/parser v0.0.0-20160622100904-31edd927e5b1/go.mod h1:2B43mz36vGZNZEwkWi8ayRSSUXLfjL8OkbzwW4NcPMM= github.com/cznic/sortutil v0.0.0-20181122101858-f5f958428db8 h1:LpMLYGyy67BoAFGda1NeOBQwqlv7nUXpm+rIVHGxZZ4= github.com/cznic/sortutil v0.0.0-20181122101858-f5f958428db8/go.mod h1:q2w6Bg5jeox1B+QkJ6Wp/+Vn0G/bo3f1uY7Fn3vivIQ= github.com/cznic/strutil v0.0.0-20171016134553-529a34b1c186 h1:0rkFMAbn5KBKNpJyHQ6Prb95vIKanmAe62KxsrN+sqA= github.com/cznic/strutil v0.0.0-20171016134553-529a34b1c186/go.mod h1:AHHPPPXTw0h6pVabbcbyGRK1DckRn7r/STdZEeIDzZc= -github.com/cznic/y v0.0.0-20170802143616-045f81c6662a h1:N2rDAvHuM46OGscJkGX4Dw4BBqZgg6mGNGLYs5utVVo= -github.com/cznic/y v0.0.0-20170802143616-045f81c6662a/go.mod h1:1rk5VM7oSnA4vjp+hrLQ3HWHa+Y4yPCa3/CsJrcNnvs= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -53,6 +47,7 @@ go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.18.1 h1:CSUJ2mjFszzEWt4CdKISEuChVIXGBn3lAPwkRGyVrc4= go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/exp v0.0.0-20181106170214-d68db9428509/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= @@ -79,3 +74,22 @@ gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +modernc.org/fileutil v1.0.0/go.mod h1:JHsWpkrk/CnVV1H/eGlFf85BEpfkrp56ro8nojIq9Q8= +modernc.org/golex v1.0.1 h1:EYKY1a3wStt0RzHaH8mdSRNg78Ub0OHxYfCRWw35YtM= +modernc.org/golex v1.0.1/go.mod h1:QCA53QtsT1NdGkaZZkF5ezFwk4IXh4BGNafAARTC254= +modernc.org/lex v1.0.0/go.mod h1:G6rxMTy3cH2iA0iXL/HRRv4Znu8MK4higxph/lE7ypk= +modernc.org/lexer v1.0.0/go.mod h1:F/Dld0YKYdZCLQ7bD0USbWL4YKCyTDRDHiDTOs0q0vk= +modernc.org/mathutil v1.0.0/go.mod h1:wU0vUrJsVWBZ4P6e7xtFJEhFSNsfRLJ8H458uRjg03k= +modernc.org/mathutil v1.4.1 h1:ij3fYGe8zBF4Vu+g0oT7mB06r8sqGWKuJu1yXeR4by8= +modernc.org/mathutil v1.4.1/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= +modernc.org/parser v1.0.0/go.mod h1:H20AntYJ2cHHL6MHthJ8LZzXCdDCHMWt1KZXtIMjejA= +modernc.org/parser v1.0.2 h1:/qHLDn1ezrcRk9/XbErYp84bPPM4+w0kIDuvMdRk6Vc= +modernc.org/parser v1.0.2/go.mod h1:TXNq3HABP3HMaqLK7brD1fLA/LfN0KS6JxZn71QdDqs= +modernc.org/scanner v1.0.1/go.mod h1:OIzD2ZtjYk6yTuyqZr57FmifbM9fIH74SumloSsajuE= +modernc.org/sortutil v1.0.0 h1:SUTM1sCR0Ldpv7dbB/KCPC2zHHsZ1KrSkhmGmmV22CQ= +modernc.org/sortutil v1.0.0/go.mod h1:1QO0q8IlIlmjBIwm6t/7sof874+xCfZouyqZMLIAtxM= +modernc.org/strutil v1.0.0/go.mod h1:lstksw84oURvj9y3tn8lGvRxyRC1S2+g5uuIzNfIOBs= +modernc.org/strutil v1.1.0 h1:+1/yCzZxY2pZwwrsbH+4T7BQMoLQ9QiBshRC9eicYsc= +modernc.org/strutil v1.1.0/go.mod h1:lstksw84oURvj9y3tn8lGvRxyRC1S2+g5uuIzNfIOBs= +modernc.org/y v1.0.1 h1:+QT+MtLkwkvLkh3fYQq+YD5vw2s5paVE73jdl5R/Py8= +modernc.org/y v1.0.1/go.mod h1:Ho86I+LVHEI+LYXoUKlmOMAM1JTXOCfj8qi1T8PsClE= diff --git a/parser/goyacc/format_yacc.go b/parser/goyacc/format_yacc.go index a814c2b5519fb..209c7acafc699 100644 --- a/parser/goyacc/format_yacc.go +++ b/parser/goyacc/format_yacc.go @@ -23,10 +23,10 @@ import ( "regexp" "strings" - parser "github.com/cznic/parser/yacc" "github.com/cznic/strutil" "github.com/pingcap/errors" "github.com/pingcap/tidb/parser/format" + parser "modernc.org/parser/yacc" ) func Format(inputFilename string, goldenFilename string) (err error) { diff --git a/parser/goyacc/main.go b/parser/goyacc/main.go index 4e9a08343dcec..1b8fae47cd756 100644 --- a/parser/goyacc/main.go +++ b/parser/goyacc/main.go @@ -142,10 +142,10 @@ import ( "strings" "github.com/cznic/mathutil" - parser "github.com/cznic/parser/yacc" "github.com/cznic/sortutil" "github.com/cznic/strutil" - "github.com/cznic/y" + parser "modernc.org/parser/yacc" + "modernc.org/y" ) var ( @@ -528,6 +528,7 @@ type %[1]sLexer interface { Lex(lval *%[1]sSymType) int Errorf(format string, a ...interface{}) error AppendError(err error) + AppendWarn(err error) Errors() (warns []error, errs []error) } diff --git a/parser/hintparser.go b/parser/hintparser.go index 71ead763fee73..146b3f33bace2 100644 --- a/parser/hintparser.go +++ b/parser/hintparser.go @@ -842,6 +842,7 @@ type yyhintLexer interface { Lex(lval *yyhintSymType) int Errorf(format string, a ...interface{}) error AppendError(err error) + AppendWarn(err error) Errors() (warns []error, errs []error) } diff --git a/parser/hintparserimpl.go b/parser/hintparserimpl.go index 5bd91a245092f..2faf988cdbb75 100644 --- a/parser/hintparserimpl.go +++ b/parser/hintparserimpl.go @@ -38,7 +38,7 @@ type hintScanner struct { func (hs *hintScanner) Errorf(format string, args ...interface{}) error { inner := hs.Scanner.Errorf(format, args...) - return ErrWarnOptimizerHintParseError.GenWithStackByArgs(inner) + return ErrParse.GenWithStackByArgs("Optimizer hint syntax error at", inner) } func (hs *hintScanner) Lex(lval *yyhintSymType) int { diff --git a/parser/lexer.go b/parser/lexer.go index 9a13812eb326b..94358fe51a962 100644 --- a/parser/lexer.go +++ b/parser/lexer.go @@ -40,7 +40,7 @@ type Scanner struct { r reader buf bytes.Buffer - encoding charset.Encoding + encoding *charset.Encoding errs []error warns []error @@ -137,6 +137,14 @@ func (s *Scanner) AppendError(err error) { s.errs = append(s.errs, err) } +// AppendWarn sets warning into scanner. +func (s *Scanner) AppendWarn(err error) { + if err == nil { + return + } + s.warns = append(s.warns, err) +} + func (s *Scanner) tryDecodeToUTF8String(sql string) string { utf8Lit, err := s.encoding.DecodeString(sql) if err != nil { diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 622b2c460730e..2dcda6cb03861 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -111,6 +111,7 @@ var actionMap = map[ActionType]string{ ActionModifyColumn: "modify column", ActionRebaseAutoID: "rebase auto_increment ID", ActionRenameTable: "rename table", + ActionRenameTables: "rename tables", ActionSetDefaultValue: "set default value", ActionShardRowID: "shard row ID", ActionModifyTableComment: "modify table comment", @@ -144,12 +145,15 @@ var actionMap = map[ActionType]string{ ActionAlterCheckConstraint: "alter check constraint", ActionDropIndexes: "drop multi-indexes", ActionAlterTableAttributes: "alter table attributes", + ActionAlterTablePartitionPolicy: "alter table partition policy", ActionAlterTablePartitionAttributes: "alter table partition attributes", ActionCreatePlacementPolicy: "create placement policy", ActionAlterPlacementPolicy: "alter placement policy", ActionDropPlacementPolicy: "drop placement policy", ActionModifySchemaDefaultPlacement: "modify schema default placement", - ActionAlterCacheTable: "alter cache table", + ActionAlterTablePlacement: "alter table placement", + ActionAlterCacheTable: "alter table cache", + ActionAlterNoCacheTable: "alter table nocache", ActionAlterTableStatsOptions: "alter table statistics options", // `ActionAlterTableAlterPartition` is removed and will never be used. diff --git a/parser/model/model_test.go b/parser/model/model_test.go index bbefb05b4c38e..8439c1ce0fcdf 100644 --- a/parser/model/model_test.go +++ b/parser/model/model_test.go @@ -284,6 +284,7 @@ func TestString(t *testing.T) { {ActionTruncateTable, "truncate table"}, {ActionModifyColumn, "modify column"}, {ActionRenameTable, "rename table"}, + {ActionRenameTables, "rename tables"}, {ActionSetDefaultValue, "set default value"}, {ActionCreateSchema, "create schema"}, {ActionDropSchema, "drop schema"}, @@ -297,6 +298,9 @@ func TestString(t *testing.T) { {ActionDropColumns, "drop multi-columns"}, {ActionModifySchemaCharsetAndCollate, "modify schema charset and collate"}, {ActionDropIndexes, "drop multi-indexes"}, + {ActionAlterTablePlacement, "alter table placement"}, + {ActionAlterTablePartitionPolicy, "alter table partition policy"}, + {ActionAlterNoCacheTable, "alter table nocache"}, } for _, v := range acts { diff --git a/parser/mysql/const.go b/parser/mysql/const.go index 23fdae95e7032..81be68bf788f6 100644 --- a/parser/mysql/const.go +++ b/parser/mysql/const.go @@ -589,7 +589,9 @@ func (n *PriorityEnum) Restore(ctx *format.RestoreCtx) error { return nil } -// PrimaryKeyName defines primary key name. const ( + // PrimaryKeyName defines primary key name. PrimaryKeyName = "PRIMARY" + // DefaultDecimal defines the default decimal value when the value out of range. + DefaultDecimal = "99999999999999999999999999999999999999999999999999999999999999999" ) diff --git a/parser/parser.go b/parser/parser.go index fd74b978614f2..dea92041d00b2 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -814,13 +814,13 @@ const ( zerofill = 57571 yyMaxDepth = 200 - yyTabOfs = -2448 + yyTabOfs = -2449 ) var ( yyXLAT = map[int]int{ - 57344: 0, // $end (2160x) - 59: 1, // ';' (2159x) + 57344: 0, // $end (2161x) + 59: 1, // ';' (2160x) 57802: 2, // remove (1835x) 57803: 3, // reorganize (1835x) 57625: 4, // comment (1771x) @@ -835,17 +835,17 @@ var ( 57613: 13, // charsetKwd (1607x) 57775: 14, // password (1603x) 58025: 15, // regions (1599x) - 57918: 16, // constraints (1592x) - 57929: 17, // followerConstraints (1592x) - 57930: 18, // followers (1592x) - 57940: 19, // leaderConstraints (1592x) - 57942: 20, // learnerConstraints (1592x) - 57943: 21, // learners (1592x) - 57952: 22, // primaryRegion (1592x) - 57957: 23, // schedule (1592x) - 57987: 24, // voterConstraints (1592x) - 57988: 25, // voters (1592x) - 57948: 26, // placement (1591x) + 57948: 16, // placement (1593x) + 57918: 17, // constraints (1592x) + 57929: 18, // followerConstraints (1592x) + 57930: 19, // followers (1592x) + 57940: 20, // leaderConstraints (1592x) + 57942: 21, // learnerConstraints (1592x) + 57943: 22, // learners (1592x) + 57952: 23, // primaryRegion (1592x) + 57957: 24, // schedule (1592x) + 57987: 25, // voterConstraints (1592x) + 57988: 26, // voters (1592x) 57615: 27, // checksum (1589x) 57662: 28, // encryption (1572x) 57714: 29, // keyBlockSize (1571x) @@ -901,7 +901,7 @@ var ( 57881: 79, // tikvImporter (1467x) 57889: 80, // truncate (1464x) 57751: 81, // no (1463x) - 57859: 82, // start (1459x) + 57859: 82, // start (1460x) 57608: 83, // cache (1458x) 57752: 84, // nocache (1457x) 57646: 85, // cycle (1456x) @@ -971,23 +971,23 @@ var ( 57766: 149, // only (1429x) 57897: 150, // value (1429x) 57599: 151, // binding (1428x) - 57663: 152, // end (1428x) - 57933: 153, // next_row_id (1428x) - 57781: 154, // policy (1428x) - 57951: 155, // predicate (1428x) - 57877: 156, // temporary (1428x) - 57890: 157, // unbounded (1428x) - 57895: 158, // user (1428x) - 57626: 159, // commit (1427x) - 57690: 160, // global (1427x) - 57346: 161, // identifier (1427x) - 57763: 162, // offset (1427x) - 57784: 163, // prepare (1427x) - 57816: 164, // role (1427x) - 57817: 165, // rollback (1427x) - 57894: 166, // unknown (1427x) - 57907: 167, // wait (1427x) - 57597: 168, // begin (1426x) + 57626: 152, // commit (1428x) + 57663: 153, // end (1428x) + 57933: 154, // next_row_id (1428x) + 57781: 155, // policy (1428x) + 57951: 156, // predicate (1428x) + 57817: 157, // rollback (1428x) + 57877: 158, // temporary (1428x) + 57890: 159, // unbounded (1428x) + 57895: 160, // user (1428x) + 57597: 161, // begin (1427x) + 57690: 162, // global (1427x) + 57346: 163, // identifier (1427x) + 57763: 164, // offset (1427x) + 57784: 165, // prepare (1427x) + 57816: 166, // role (1427x) + 57894: 167, // unknown (1427x) + 57907: 168, // wait (1427x) 57606: 169, // btree (1426x) 57648: 170, // datetimeType (1426x) 57649: 171, // dateType (1426x) @@ -1022,63 +1022,63 @@ var ( 57735: 200, // mb (1425x) 57742: 201, // mode (1425x) 57748: 202, // never (1425x) - 57780: 203, // plugins (1425x) - 57788: 204, // processlist (1425x) - 57799: 205, // recover (1425x) - 57804: 206, // repair (1425x) - 57805: 207, // repeatable (1425x) - 57833: 208, // session (1425x) - 58010: 209, // statistics (1425x) - 57868: 210, // subpartitions (1425x) - 58019: 211, // tidb (1425x) - 57882: 212, // timestampType (1425x) - 57904: 213, // without (1425x) - 57989: 214, // admin (1424x) - 57595: 215, // backup (1424x) - 57601: 216, // binlog (1424x) - 57603: 217, // block (1424x) - 57604: 218, // booleanType (1424x) - 57990: 219, // buckets (1424x) - 57993: 220, // cardinality (1424x) - 57612: 221, // chain (1424x) - 57619: 222, // clientErrorsSummary (1424x) - 57994: 223, // cmSketch (1424x) - 57620: 224, // coalesce (1424x) - 57628: 225, // compact (1424x) - 57629: 226, // compressed (1424x) - 57635: 227, // context (1424x) - 57917: 228, // copyKwd (1424x) - 57996: 229, // correlation (1424x) - 57636: 230, // cpu (1424x) - 57651: 231, // deallocate (1424x) - 57998: 232, // dependency (1424x) - 57654: 233, // directory (1424x) - 57656: 234, // discard (1424x) - 57657: 235, // disk (1424x) - 57658: 236, // do (1424x) - 58000: 237, // drainer (1424x) - 57673: 238, // exchange (1424x) - 57675: 239, // execute (1424x) - 57676: 240, // expansion (1424x) - 57927: 241, // flashback (1424x) - 57689: 242, // general (1424x) - 57693: 243, // help (1424x) - 57694: 244, // histogram (1424x) - 57696: 245, // hosts (1424x) - 57934: 246, // inplace (1424x) - 57935: 247, // instant (1424x) - 57710: 248, // ipc (1424x) - 58002: 249, // job (1424x) - 58001: 250, // jobs (1424x) - 57715: 251, // labels (1424x) - 57724: 252, // locked (1424x) - 57743: 253, // modify (1424x) - 57749: 254, // next (1424x) - 58003: 255, // nodeID (1424x) - 58004: 256, // nodeState (1424x) - 57761: 257, // nulls (1424x) - 57770: 258, // pageSym (1424x) - 57949: 259, // plan (1424x) + 57949: 203, // plan (1425x) + 57780: 204, // plugins (1425x) + 57788: 205, // processlist (1425x) + 57799: 206, // recover (1425x) + 57804: 207, // repair (1425x) + 57805: 208, // repeatable (1425x) + 57833: 209, // session (1425x) + 58010: 210, // statistics (1425x) + 57868: 211, // subpartitions (1425x) + 58019: 212, // tidb (1425x) + 57882: 213, // timestampType (1425x) + 57904: 214, // without (1425x) + 57989: 215, // admin (1424x) + 57595: 216, // backup (1424x) + 57601: 217, // binlog (1424x) + 57603: 218, // block (1424x) + 57604: 219, // booleanType (1424x) + 57990: 220, // buckets (1424x) + 57993: 221, // cardinality (1424x) + 57612: 222, // chain (1424x) + 57619: 223, // clientErrorsSummary (1424x) + 57994: 224, // cmSketch (1424x) + 57620: 225, // coalesce (1424x) + 57628: 226, // compact (1424x) + 57629: 227, // compressed (1424x) + 57635: 228, // context (1424x) + 57917: 229, // copyKwd (1424x) + 57996: 230, // correlation (1424x) + 57636: 231, // cpu (1424x) + 57651: 232, // deallocate (1424x) + 57998: 233, // dependency (1424x) + 57654: 234, // directory (1424x) + 57656: 235, // discard (1424x) + 57657: 236, // disk (1424x) + 57658: 237, // do (1424x) + 58000: 238, // drainer (1424x) + 57673: 239, // exchange (1424x) + 57675: 240, // execute (1424x) + 57676: 241, // expansion (1424x) + 57927: 242, // flashback (1424x) + 57689: 243, // general (1424x) + 57693: 244, // help (1424x) + 57694: 245, // histogram (1424x) + 57696: 246, // hosts (1424x) + 57934: 247, // inplace (1424x) + 57935: 248, // instant (1424x) + 57710: 249, // ipc (1424x) + 58002: 250, // job (1424x) + 58001: 251, // jobs (1424x) + 57715: 252, // labels (1424x) + 57724: 253, // locked (1424x) + 57743: 254, // modify (1424x) + 57749: 255, // next (1424x) + 58003: 256, // nodeID (1424x) + 58004: 257, // nodeState (1424x) + 57761: 258, // nulls (1424x) + 57770: 259, // pageSym (1424x) 58007: 260, // pump (1424x) 57792: 261, // purge (1424x) 57798: 262, // rebuild (1424x) @@ -1277,9 +1277,9 @@ var ( 57984: 455, // varSamp (1422x) 57986: 456, // voter (1422x) 57903: 457, // weightString (1422x) - 57488: 458, // on (1368x) - 40: 459, // '(' (1283x) - 57568: 460, // with (1183x) + 57488: 458, // on (1369x) + 40: 459, // '(' (1284x) + 57568: 460, // with (1184x) 57349: 461, // stringLit (1168x) 58074: 462, // not2 (1155x) 57481: 463, // not (1100x) @@ -1303,21 +1303,21 @@ var ( 57443: 481, // into (900x) 58063: 482, // eq (896x) 57469: 483, // lock (896x) - 57557: 484, // values (893x) + 57557: 484, // values (894x) 57421: 485, // force (891x) 57377: 486, // charType (888x) 57423: 487, // from (887x) 57417: 488, // fetch (886x) 57565: 489, // where (884x) 57493: 490, // order (882x) - 57363: 491, // and (868x) - 57511: 492, // replace (867x) + 57511: 491, // replace (868x) + 57363: 492, // and (867x) 58058: 493, // intLit (856x) - 57492: 494, // or (845x) - 57354: 495, // andand (844x) - 57779: 496, // pipesAsOr (844x) - 57569: 497, // xor (844x) - 57522: 498, // set (839x) + 57492: 494, // or (844x) + 57354: 495, // andand (843x) + 57779: 496, // pipesAsOr (843x) + 57569: 497, // xor (843x) + 57522: 498, // set (840x) 57427: 499, // group (816x) 57533: 500, // straightJoin (812x) 57567: 501, // window (804x) @@ -1372,9 +1372,9 @@ var ( 57507: 550, // regexpKwd (733x) 57516: 551, // rlike (733x) 57434: 552, // ifKwd (730x) - 57534: 553, // tableKwd (717x) + 57534: 553, // tableKwd (718x) 57350: 554, // singleAtIdentifier (712x) - 57446: 555, // insert (710x) + 57446: 555, // insert (711x) 57389: 556, // currentUser (708x) 57416: 557, // falseKwd (706x) 57545: 558, // trueKwd (706x) @@ -1451,7 +1451,7 @@ var ( 57546: 629, // unique (681x) 57381: 630, // constraint (679x) 57506: 631, // references (676x) - 57521: 632, // selectKwd (674x) + 57521: 632, // selectKwd (675x) 57425: 633, // generated (672x) 57376: 634, // character (662x) 57437: 635, // index (644x) @@ -1460,7 +1460,7 @@ var ( 57360: 638, // all (540x) 46: 639, // '.' (531x) 57362: 640, // analyze (515x) - 57550: 641, // update (501x) + 57550: 641, // update (502x) 58066: 642, // jss (499x) 58067: 643, // juss (499x) 57474: 644, // maxValue (497x) @@ -1515,7 +1515,7 @@ var ( 57539: 693, // tinyblobType (461x) 57540: 694, // tinyIntType (461x) 57541: 695, // tinytextType (461x) - 58581: 696, // SubSelect (207x) + 58581: 696, // SubSelect (208x) 58635: 697, // UserVariable (171x) 58557: 698, // SimpleIdent (170x) 58372: 699, // Literal (168x) @@ -1539,8 +1539,8 @@ var ( 58466: 717, // PredicateExpr (130x) 58152: 718, // BoolPri (127x) 58264: 719, // Expression (127x) - 58684: 720, // logAnd (97x) - 58685: 721, // logOr (97x) + 58684: 720, // logAnd (96x) + 58685: 721, // logOr (96x) 58391: 722, // NUM (96x) 58254: 723, // EqOpt (86x) 58594: 724, // TableName (75x) @@ -1550,28 +1550,28 @@ var ( 57571: 728, // zerofill (45x) 58174: 729, // ColumnName (40x) 58363: 730, // LengthNum (40x) - 57400: 731, // deleteKwd (38x) + 57400: 731, // deleteKwd (39x) 57404: 732, // distinct (36x) 57405: 733, // distinctRow (36x) 58674: 734, // WindowingClause (35x) 57399: 735, // delayed (33x) 57430: 736, // highPriority (33x) 57472: 737, // lowPriority (33x) - 58512: 738, // SelectStmt (28x) - 58513: 739, // SelectStmtBasic (28x) - 58515: 740, // SelectStmtFromDualTable (28x) - 58516: 741, // SelectStmtFromTable (28x) - 58532: 742, // SetOprClause (28x) - 57353: 743, // hintComment (27x) - 58533: 744, // SetOprClauseList (27x) - 58536: 745, // SetOprStmtWithLimitOrderBy (27x) - 58537: 746, // SetOprStmtWoutLimitOrderBy (27x) + 58512: 738, // SelectStmt (29x) + 58513: 739, // SelectStmtBasic (29x) + 58515: 740, // SelectStmtFromDualTable (29x) + 58516: 741, // SelectStmtFromTable (29x) + 58532: 742, // SetOprClause (29x) + 58533: 743, // SetOprClauseList (28x) + 58536: 744, // SetOprStmtWithLimitOrderBy (28x) + 58537: 745, // SetOprStmtWoutLimitOrderBy (28x) + 57353: 746, // hintComment (27x) 58275: 747, // FieldLen (26x) 58352: 748, // Int64Num (26x) - 58432: 749, // OptWindowingClause (24x) - 58525: 750, // SelectStmtWithClause (24x) - 58535: 751, // SetOprStmt (24x) - 58675: 752, // WithClause (24x) + 58525: 749, // SelectStmtWithClause (25x) + 58535: 750, // SetOprStmt (25x) + 58675: 751, // WithClause (25x) + 58432: 752, // OptWindowingClause (24x) 58437: 753, // OrderBy (23x) 58519: 754, // SelectStmtLimit (23x) 57527: 755, // sqlBigResult (23x) @@ -1582,39 +1582,39 @@ var ( 58637: 760, // Username (20x) 58265: 761, // ExpressionList (17x) 58461: 762, // PlacementPolicyOption (17x) - 58321: 763, // IfExists (16x) - 58459: 764, // PlacementOption (16x) - 57537: 765, // terminated (16x) - 58629: 766, // UpdateStmtNoWith (16x) - 58230: 767, // DeleteWithoutUsingStmt (15x) + 58629: 763, // UpdateStmtNoWith (17x) + 58230: 764, // DeleteWithoutUsingStmt (16x) + 58321: 765, // IfExists (16x) + 58459: 766, // PlacementOption (16x) + 57537: 767, // terminated (16x) 58232: 768, // DistinctKwd (15x) 58322: 769, // IfNotExists (15x) - 58417: 770, // OptFieldLen (15x) - 58233: 771, // DistinctOpt (14x) - 57411: 772, // enclosed (14x) - 58349: 773, // InsertIntoStmt (14x) - 58448: 774, // PartitionNameList (14x) - 58487: 775, // ReplaceIntoStmt (14x) - 58628: 776, // UpdateStmt (14x) + 58349: 770, // InsertIntoStmt (15x) + 58417: 771, // OptFieldLen (15x) + 58487: 772, // ReplaceIntoStmt (15x) + 58628: 773, // UpdateStmt (15x) + 58233: 774, // DistinctOpt (14x) + 57411: 775, // enclosed (14x) + 58448: 776, // PartitionNameList (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) + 58229: 783, // DeleteWithUsingStmt (12x) + 58263: 784, // ExprOrDefault (12x) + 58357: 785, // JoinTable (12x) + 58411: 786, // OptBinary (12x) + 58503: 787, // RolenameComposed (12x) + 58591: 788, // TableFactor (12x) + 58604: 789, // TableRef (12x) + 58124: 790, // AnalyzeOptionListOpt (11x) + 58228: 791, // DeleteFromStmt (11x) + 58292: 792, // FromOrIn (11x) + 58618: 793, // TimestampUnit (11x) + 58163: 794, // CharsetName (10x) + 58175: 795, // ColumnNameList (10x) 58396: 796, // NotSym (10x) 58438: 797, // OrderByOptional (10x) 58440: 798, // PartDefOption (10x) @@ -1623,17 +1623,17 @@ var ( 58215: 801, // DBName (9x) 58224: 802, // DefaultFalseDistinctOpt (9x) 58358: 803, // JoinType (9x) - 57482: 804, // noWriteToBinLog (9x) - 58401: 805, // NumLiteral (9x) - 58502: 806, // Rolename (9x) - 58497: 807, // RoleNameString (9x) - 58120: 808, // AlterTableStmt (8x) - 58214: 809, // CrossOpt (8x) - 58255: 810, // EqOrAssignmentEq (8x) - 58266: 811, // ExpressionListOpt (8x) - 58343: 812, // IndexPartSpecification (8x) - 58359: 813, // KeyOrIndex (8x) - 57466: 814, // load (8x) + 57466: 804, // load (9x) + 57482: 805, // noWriteToBinLog (9x) + 58401: 806, // NumLiteral (9x) + 58502: 807, // Rolename (9x) + 58497: 808, // RoleNameString (9x) + 58120: 809, // AlterTableStmt (8x) + 58214: 810, // CrossOpt (8x) + 58255: 811, // EqOrAssignmentEq (8x) + 58266: 812, // ExpressionListOpt (8x) + 58343: 813, // IndexPartSpecification (8x) + 58359: 814, // KeyOrIndex (8x) 58520: 815, // SelectStmtLimitOpt (8x) 58617: 816, // TimeUnit (8x) 58649: 817, // VariableName (8x) @@ -1646,78 +1646,78 @@ var ( 58470: 824, // Priority (7x) 58507: 825, // RowFormat (7x) 58510: 826, // RowValue (7x) - 58541: 827, // ShowDatabaseNameOpt (7x) - 58601: 828, // TableOption (7x) - 57562: 829, // varying (7x) - 57380: 830, // column (6x) - 58169: 831, // ColumnDef (6x) - 58217: 832, // DatabaseOption (6x) - 58220: 833, // DatabaseSym (6x) - 58257: 834, // EscapedTableRef (6x) - 58262: 835, // ExplainableStmt (6x) - 58279: 836, // FieldTerminator (6x) - 57426: 837, // grant (6x) - 58326: 838, // IgnoreOptional (6x) - 58335: 839, // IndexInvisible (6x) - 58340: 840, // IndexNameList (6x) - 58346: 841, // IndexType (6x) - 58449: 842, // PartitionNameListOpt (6x) - 57508: 843, // release (6x) - 58504: 844, // RolenameList (6x) - 58530: 845, // SetExpr (6x) + 58530: 827, // SetExpr (7x) + 58541: 828, // ShowDatabaseNameOpt (7x) + 58601: 829, // TableOption (7x) + 57562: 830, // varying (7x) + 57380: 831, // column (6x) + 58169: 832, // ColumnDef (6x) + 58217: 833, // DatabaseOption (6x) + 58220: 834, // DatabaseSym (6x) + 58257: 835, // EscapedTableRef (6x) + 58262: 836, // ExplainableStmt (6x) + 58279: 837, // FieldTerminator (6x) + 57426: 838, // grant (6x) + 58326: 839, // IgnoreOptional (6x) + 58335: 840, // IndexInvisible (6x) + 58340: 841, // IndexNameList (6x) + 58346: 842, // IndexType (6x) + 58449: 843, // PartitionNameListOpt (6x) + 57508: 844, // release (6x) + 58504: 845, // RolenameList (6x) 57523: 846, // show (6x) 58599: 847, // TableOptimizerHints (6x) 58638: 848, // UsernameList (6x) 58676: 849, // WithClustered (6x) 58104: 850, // AlgorithmClause (5x) - 58156: 851, // ByItem (5x) - 58168: 852, // CollationName (5x) - 58172: 853, // ColumnKeywordOpt (5x) - 58277: 854, // FieldOpt (5x) - 58278: 855, // FieldOpts (5x) - 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) + 58145: 851, // BeginTransactionStmt (5x) + 58156: 852, // ByItem (5x) + 58168: 853, // CollationName (5x) + 58172: 854, // ColumnKeywordOpt (5x) + 58188: 855, // CommitStmt (5x) + 58277: 856, // FieldOpt (5x) + 58278: 857, // FieldOpts (5x) + 58318: 858, // IdentList (5x) + 58338: 859, // IndexName (5x) + 58341: 860, // IndexOption (5x) + 58342: 861, // IndexOptionList (5x) + 57438: 862, // infile (5x) + 58368: 863, // LimitOption (5x) + 58376: 864, // LoadDataStmt (5x) + 58380: 865, // LockClause (5x) + 58413: 866, // OptCharsetWithOptBinary (5x) + 58424: 867, // OptNullTreatment (5x) + 58464: 868, // PolicyName (5x) + 58471: 869, // PriorityOpt (5x) + 58506: 870, // RollbackStmt (5x) + 58511: 871, // SelectLockOpt (5x) + 58518: 872, // SelectStmtIntoOption (5x) + 58540: 873, // SetStmt (5x) + 58605: 874, // TableRefs (5x) + 58631: 875, // UserSpec (5x) + 58130: 876, // Assignment (4x) + 58136: 877, // AuthString (4x) + 58147: 878, // BindableStmt (4x) + 58137: 879, // BRIEBooleanOptionName (4x) + 58138: 880, // BRIEIntegerOptionName (4x) + 58139: 881, // BRIEKeywordOptionName (4x) + 58140: 882, // BRIEOption (4x) + 58141: 883, // BRIEOptions (4x) + 58143: 884, // BRIEStringOptionName (4x) + 58157: 885, // ByList (4x) + 58161: 886, // Char (4x) + 58192: 887, // ConfigItemName (4x) + 58196: 888, // Constraint (4x) + 58286: 889, // FloatOpt (4x) + 58347: 890, // IndexTypeName (4x) + 57490: 891, // option (4x) + 58429: 892, // OptWild (4x) + 57494: 893, // outer (4x) + 58465: 894, // Precision (4x) + 58479: 895, // ReferDef (4x) + 58493: 896, // RestrictOrCascadeOpt (4x) + 58509: 897, // RowStmt (4x) + 58526: 898, // SequenceOption (4x) 57532: 899, // statsExtended (4x) 58586: 900, // TableAsName (4x) 58587: 901, // TableAsNameOpt (4x) @@ -1771,183 +1771,183 @@ var ( 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) + 58620: 952, // TraceableStmt (3x) + 58622: 953, // TransactionChars (3x) + 57544: 954, // trigger (3x) + 57548: 955, // unlock (3x) + 57551: 956, // usage (3x) + 58642: 957, // ValuesList (3x) + 58644: 958, // ValuesStmtList (3x) + 58640: 959, // ValueSym (3x) + 58647: 960, // VariableAssignment (3x) + 58667: 961, // WindowFrameStart (3x) + 58103: 962, // AdminStmt (2x) + 58105: 963, // AllColumnsOrPredicateColumnsOpt (2x) + 58107: 964, // AlterDatabaseStmt (2x) + 58108: 965, // AlterImportStmt (2x) + 58109: 966, // AlterInstanceStmt (2x) + 58110: 967, // AlterOrderItem (2x) + 58112: 968, // AlterPolicyStmt (2x) + 58113: 969, // AlterSequenceOption (2x) + 58115: 970, // AlterSequenceStmt (2x) + 58117: 971, // AlterTableSpec (2x) + 58121: 972, // AlterUserStmt (2x) + 58122: 973, // AnalyzeOption (2x) + 58125: 974, // AnalyzeTableStmt (2x) + 58148: 975, // BinlogStmt (2x) + 58142: 976, // BRIEStmt (2x) + 58144: 977, // BRIETables (2x) + 57372: 978, // call (2x) + 58158: 979, // CallStmt (2x) + 58159: 980, // CastType (2x) + 58160: 981, // ChangeStmt (2x) + 58166: 982, // CheckConstraintKeyword (2x) + 58176: 983, // ColumnNameListOpt (2x) + 58179: 984, // ColumnNameOrUserVariable (2x) + 58182: 985, // ColumnOptionList (2x) + 58183: 986, // ColumnOptionListOpt (2x) + 58185: 987, // ColumnSetValue (2x) + 58191: 988, // CompletionTypeWithinTransaction (2x) + 58193: 989, // ConnectionOption (2x) + 58195: 990, // ConnectionOptions (2x) + 58199: 991, // CreateBindingStmt (2x) + 58200: 992, // CreateDatabaseStmt (2x) + 58201: 993, // CreateImportStmt (2x) + 58202: 994, // CreateIndexStmt (2x) + 58203: 995, // CreatePolicyStmt (2x) + 58204: 996, // CreateRoleStmt (2x) + 58206: 997, // CreateSequenceStmt (2x) + 58207: 998, // CreateStatisticsStmt (2x) + 58208: 999, // CreateTableOptionListOpt (2x) + 58211: 1000, // CreateUserStmt (2x) + 58213: 1001, // CreateViewStmt (2x) + 57392: 1002, // databases (2x) + 58222: 1003, // DeallocateStmt (2x) + 58223: 1004, // DeallocateSym (2x) + 57403: 1005, // describe (2x) + 58234: 1006, // DoStmt (2x) + 58235: 1007, // DropBindingStmt (2x) + 58236: 1008, // DropDatabaseStmt (2x) + 58237: 1009, // DropImportStmt (2x) + 58238: 1010, // DropIndexStmt (2x) + 58239: 1011, // DropPolicyStmt (2x) + 58240: 1012, // DropRoleStmt (2x) + 58241: 1013, // DropSequenceStmt (2x) + 58242: 1014, // DropStatisticsStmt (2x) + 58243: 1015, // DropStatsStmt (2x) + 58244: 1016, // DropTableStmt (2x) + 58245: 1017, // DropUserStmt (2x) + 58246: 1018, // DropViewStmt (2x) + 58247: 1019, // DuplicateOpt (2x) + 58249: 1020, // EmptyStmt (2x) + 58250: 1021, // EncryptionOpt (2x) + 58252: 1022, // EnforcedOrNotOpt (2x) + 58256: 1023, // ErrorHandling (2x) + 58258: 1024, // ExecuteStmt (2x) + 58260: 1025, // ExplainStmt (2x) + 58261: 1026, // ExplainSym (2x) + 58270: 1027, // Field (2x) + 58273: 1028, // FieldItem (2x) + 58280: 1029, // Fields (2x) + 58284: 1030, // FlashbackTableStmt (2x) + 58289: 1031, // FlushStmt (2x) + 58295: 1032, // FuncDatetimePrecList (2x) + 58296: 1033, // FuncDatetimePrecListOpt (2x) + 58309: 1034, // GrantProxyStmt (2x) + 58310: 1035, // GrantRoleStmt (2x) + 58311: 1036, // GrantStmt (2x) + 58313: 1037, // HandleRange (2x) + 58315: 1038, // HashString (2x) + 58317: 1039, // HelpStmt (2x) + 58329: 1040, // IndexAdviseStmt (2x) + 58331: 1041, // IndexHintList (2x) + 58332: 1042, // IndexHintListOpt (2x) + 58337: 1043, // IndexLockAndAlgorithmOpt (2x) + 58350: 1044, // InsertValues (2x) + 58354: 1045, // IntoOpt (2x) + 58360: 1046, // KeyOrIndexOpt (2x) + 57456: 1047, // kill (2x) + 58361: 1048, // KillOrKillTiDB (2x) + 58362: 1049, // KillStmt (2x) + 58367: 1050, // LimitClause (2x) + 57465: 1051, // linear (2x) + 58369: 1052, // LinearOpt (2x) + 58373: 1053, // LoadDataSetItem (2x) + 58377: 1054, // LoadStatsStmt (2x) + 58378: 1055, // LocalOpt (2x) + 58381: 1056, // LockTablesStmt (2x) + 58389: 1057, // MaxValueOrExpressionList (2x) + 58397: 1058, // NowSym (2x) + 58398: 1059, // NowSymFunc (2x) + 58399: 1060, // NowSymOptionFraction (2x) + 58400: 1061, // NumList (2x) + 58403: 1062, // ObjectType (2x) + 57487: 1063, // of (2x) + 58404: 1064, // OfTablesOpt (2x) + 58405: 1065, // OnCommitOpt (2x) + 58406: 1066, // OnDelete (2x) + 58409: 1067, // OnUpdate (2x) + 58414: 1068, // OptCollate (2x) + 58419: 1069, // OptFull (2x) + 58421: 1070, // OptInteger (2x) + 58434: 1071, // OptionalBraces (2x) + 58433: 1072, // OptionLevel (2x) + 58423: 1073, // OptLeadLagInfo (2x) + 58422: 1074, // OptLLDefault (2x) + 58439: 1075, // OuterOpt (2x) + 58444: 1076, // PartitionDefinitionList (2x) + 58445: 1077, // PartitionDefinitionListOpt (2x) + 58451: 1078, // PartitionOpt (2x) + 58453: 1079, // PasswordOpt (2x) + 58455: 1080, // PasswordOrLockOptionList (2x) + 58456: 1081, // PasswordOrLockOptions (2x) + 58460: 1082, // PlacementOptionList (2x) + 58462: 1083, // PlanReplayerStmt (2x) + 58468: 1084, // PreparedStmt (2x) + 58473: 1085, // PrivLevel (2x) + 58476: 1086, // PurgeImportStmt (2x) + 58477: 1087, // QuickOptional (2x) + 58478: 1088, // RecoverTableStmt (2x) + 58480: 1089, // ReferOpt (2x) + 58482: 1090, // RegexpSym (2x) + 58483: 1091, // RenameTableStmt (2x) + 58484: 1092, // RenameUserStmt (2x) + 58486: 1093, // RepeatableOpt (2x) + 58492: 1094, // RestartStmt (2x) + 58494: 1095, // ResumeImportStmt (2x) + 57514: 1096, // revoke (2x) + 58495: 1097, // RevokeRoleStmt (2x) + 58496: 1098, // RevokeStmt (2x) + 58499: 1099, // RoleOrPrivElemList (2x) + 58500: 1100, // RoleSpec (2x) + 58521: 1101, // SelectStmtOpt (2x) + 58524: 1102, // SelectStmtSQLCache (2x) + 58528: 1103, // SetDefaultRoleOpt (2x) + 58529: 1104, // SetDefaultRoleStmt (2x) + 58539: 1105, // SetRoleStmt (2x) + 58542: 1106, // ShowImportStmt (2x) + 58547: 1107, // ShowProfileType (2x) + 58550: 1108, // ShowStmt (2x) + 58551: 1109, // ShowTableAliasOpt (2x) + 58553: 1110, // ShutdownStmt (2x) + 58554: 1111, // SignedLiteral (2x) + 58558: 1112, // SplitOption (2x) + 58559: 1113, // SplitRegionStmt (2x) + 58563: 1114, // Statement (2x) + 58565: 1115, // StatsOptionsOpt (2x) + 58566: 1116, // StatsPersistentVal (2x) + 58567: 1117, // StatsType (2x) + 58568: 1118, // StopImportStmt (2x) + 58575: 1119, // SubPartDefinition (2x) + 58578: 1120, // SubPartitionMethod (2x) + 58583: 1121, // Symbol (2x) + 58589: 1122, // TableElementList (2x) + 58592: 1123, // TableLock (2x) + 58596: 1124, // TableNameListOpt (2x) + 58603: 1125, // TableOrTables (2x) + 58612: 1126, // TablesTerminalSym (2x) + 58610: 1127, // TableToTable (2x) + 58614: 1128, // TextStringList (2x) 58619: 1129, // TraceStmt (2x) 58624: 1130, // TruncateTableStmt (2x) 58627: 1131, // UnlockTablesStmt (2x) @@ -2200,6 +2200,7 @@ var ( "charsetKwd", "password", "regions", + "placement", "constraints", "followerConstraints", "followers", @@ -2210,7 +2211,6 @@ var ( "schedule", "voterConstraints", "voters", - "placement", "checksum", "encryption", "keyBlockSize", @@ -2336,23 +2336,23 @@ var ( "only", "value", "binding", + "commit", "end", "next_row_id", "policy", "predicate", + "rollback", "temporary", "unbounded", "user", - "commit", + "begin", "global", "identifier", "offset", "prepare", "role", - "rollback", "unknown", "wait", - "begin", "btree", "datetimeType", "dateType", @@ -2387,6 +2387,7 @@ var ( "mb", "mode", "never", + "plan", "plugins", "processlist", "recover", @@ -2443,7 +2444,6 @@ var ( "nodeState", "nulls", "pageSym", - "plan", "pump", "purge", "rebuild", @@ -2675,8 +2675,8 @@ var ( "fetch", "where", "order", - "and", "replace", + "and", "intLit", "or", "andand", @@ -2927,16 +2927,16 @@ var ( "SelectStmtFromDualTable", "SelectStmtFromTable", "SetOprClause", - "hintComment", "SetOprClauseList", "SetOprStmtWithLimitOrderBy", "SetOprStmtWoutLimitOrderBy", + "hintComment", "FieldLen", "Int64Num", - "OptWindowingClause", "SelectStmtWithClause", "SetOprStmt", "WithClause", + "OptWindowingClause", "OrderBy", "SelectStmtLimit", "sqlBigResult", @@ -2947,39 +2947,39 @@ var ( "Username", "ExpressionList", "PlacementPolicyOption", + "UpdateStmtNoWith", + "DeleteWithoutUsingStmt", "IfExists", "PlacementOption", "terminated", - "UpdateStmtNoWith", - "DeleteWithoutUsingStmt", "DistinctKwd", "IfNotExists", + "InsertIntoStmt", "OptFieldLen", + "ReplaceIntoStmt", + "UpdateStmt", "DistinctOpt", "enclosed", - "InsertIntoStmt", "PartitionNameList", - "ReplaceIntoStmt", - "UpdateStmt", "WhereClause", "WhereClauseOptional", "DefaultKwdOpt", "escaped", "optionally", "TableNameList", + "DeleteWithUsingStmt", + "ExprOrDefault", "JoinTable", "OptBinary", "RolenameComposed", "TableFactor", "TableRef", "AnalyzeOptionListOpt", - "DeleteWithUsingStmt", - "ExprOrDefault", + "DeleteFromStmt", "FromOrIn", "TimestampUnit", "CharsetName", "ColumnNameList", - "DeleteFromStmt", "NotSym", "OrderByOptional", "PartDefOption", @@ -2988,6 +2988,7 @@ var ( "DBName", "DefaultFalseDistinctOpt", "JoinType", + "load", "noWriteToBinLog", "NumLiteral", "Rolename", @@ -2998,7 +2999,6 @@ var ( "ExpressionListOpt", "IndexPartSpecification", "KeyOrIndex", - "load", "SelectStmtLimitOpt", "TimeUnit", "VariableName", @@ -3011,6 +3011,7 @@ var ( "Priority", "RowFormat", "RowValue", + "SetExpr", "ShowDatabaseNameOpt", "TableOption", "varying", @@ -3029,15 +3030,16 @@ var ( "PartitionNameListOpt", "release", "RolenameList", - "SetExpr", "show", "TableOptimizerHints", "UsernameList", "WithClustered", "AlgorithmClause", + "BeginTransactionStmt", "ByItem", "CollationName", "ColumnKeywordOpt", + "CommitStmt", "FieldOpt", "FieldOpts", "IdentList", @@ -3046,18 +3048,20 @@ var ( "IndexOptionList", "infile", "LimitOption", + "LoadDataStmt", "LockClause", "OptCharsetWithOptBinary", "OptNullTreatment", "PolicyName", "PriorityOpt", + "RollbackStmt", "SelectLockOpt", "SelectStmtIntoOption", + "SetStmt", "TableRefs", "UserSpec", "Assignment", "AuthString", - "BeginTransactionStmt", "BindableStmt", "BRIEBooleanOptionName", "BRIEIntegerOptionName", @@ -3067,22 +3071,18 @@ var ( "BRIEStringOptionName", "ByList", "Char", - "CommitStmt", "ConfigItemName", "Constraint", "FloatOpt", "IndexTypeName", - "LoadDataStmt", "option", "OptWild", "outer", "Precision", "ReferDef", "RestrictOrCascadeOpt", - "RollbackStmt", "RowStmt", "SequenceOption", - "SetStmt", "statsExtended", "TableAsName", "TableAsNameOpt", @@ -3136,6 +3136,7 @@ var ( "TableElement", "TableNameListOpt2", "TextString", + "TraceableStmt", "TransactionChars", "trigger", "unlock", @@ -3312,7 +3313,6 @@ var ( "TablesTerminalSym", "TableToTable", "TextStringList", - "TraceableStmt", "TraceStmt", "TruncateTableStmt", "UnlockTablesStmt", @@ -3551,12 +3551,12 @@ var ( yyReductions = []struct{ xsym, components int }{ {0, 1}, {1277, 1}, - {808, 6}, - {808, 8}, - {808, 10}, - {1081, 1}, - {1081, 2}, - {1081, 3}, + {809, 6}, + {809, 8}, + {809, 10}, + {1082, 1}, + {1082, 2}, + {1082, 3}, {758, 3}, {758, 3}, {758, 3}, @@ -3568,16 +3568,16 @@ var ( {758, 3}, {758, 3}, {758, 3}, - {764, 1}, - {764, 1}, + {766, 1}, + {766, 1}, {762, 4}, {762, 4}, {762, 4}, {762, 4}, {910, 3}, {910, 3}, - {1114, 3}, - {1114, 3}, + {1115, 3}, + {1115, 3}, {1146, 1}, {1146, 2}, {1146, 4}, @@ -3585,61 +3585,61 @@ var ( {1146, 3}, {1221, 0}, {1221, 3}, - {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}, + {971, 1}, + {971, 5}, + {971, 5}, + {971, 5}, + {971, 5}, + {971, 6}, + {971, 2}, + {971, 5}, + {971, 6}, + {971, 8}, + {971, 1}, + {971, 1}, + {971, 3}, + {971, 4}, + {971, 5}, + {971, 3}, + {971, 4}, + {971, 4}, + {971, 7}, + {971, 3}, + {971, 4}, + {971, 4}, + {971, 4}, + {971, 4}, + {971, 2}, + {971, 2}, + {971, 4}, + {971, 4}, + {971, 5}, + {971, 3}, + {971, 2}, + {971, 2}, + {971, 5}, + {971, 6}, + {971, 6}, + {971, 8}, + {971, 5}, + {971, 5}, + {971, 3}, + {971, 3}, + {971, 3}, + {971, 5}, + {971, 1}, + {971, 1}, + {971, 1}, + {971, 1}, + {971, 2}, + {971, 2}, + {971, 1}, + {971, 1}, + {971, 4}, + {971, 3}, + {971, 4}, + {971, 1}, + {971, 1}, {1257, 0}, {1257, 5}, {818, 1}, @@ -3655,16 +3655,16 @@ var ( {850, 3}, {850, 3}, {850, 3}, - {862, 3}, - {862, 3}, + {865, 3}, + {865, 3}, {1142, 2}, {1142, 2}, - {813, 1}, - {813, 1}, - {1045, 0}, - {1045, 1}, - {853, 0}, - {853, 1}, + {814, 1}, + {814, 1}, + {1046, 0}, + {1046, 1}, + {854, 0}, + {854, 1}, {913, 0}, {913, 1}, {913, 2}, @@ -3672,105 +3672,105 @@ var ( {1148, 1}, {1147, 1}, {1147, 3}, - {774, 1}, - {774, 3}, + {776, 1}, + {776, 3}, {819, 0}, {819, 1}, {819, 2}, - {1120, 1}, - {1090, 3}, + {1121, 1}, + {1091, 3}, {1296, 1}, {1296, 3}, - {1126, 3}, - {1091, 3}, + {1127, 3}, + {1092, 3}, {1301, 1}, {1301, 3}, {1132, 3}, - {1087, 5}, - {1087, 3}, - {1087, 4}, - {1029, 4}, + {1088, 5}, + {1088, 3}, + {1088, 4}, + {1030, 4}, {1191, 0}, {1191, 2}, + {1113, 6}, + {1113, 8}, {1112, 6}, - {1112, 8}, - {1111, 6}, - {1111, 2}, + {1112, 2}, {1275, 0}, {1275, 2}, {1275, 1}, {1275, 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}, + {974, 5}, + {974, 6}, + {974, 7}, + {974, 7}, + {974, 8}, + {974, 9}, + {974, 8}, + {974, 7}, + {974, 6}, + {974, 8}, + {963, 0}, + {963, 2}, + {963, 2}, + {790, 0}, + {790, 2}, {1149, 1}, {1149, 3}, - {972, 2}, - {972, 2}, - {972, 3}, - {972, 3}, - {972, 2}, - {972, 2}, - {871, 3}, + {973, 2}, + {973, 2}, + {973, 3}, + {973, 3}, + {973, 2}, + {973, 2}, + {876, 3}, {909, 1}, {909, 3}, {1328, 0}, {1328, 1}, - {873, 1}, - {873, 2}, - {873, 2}, - {873, 2}, - {873, 4}, - {873, 5}, - {873, 6}, - {873, 4}, - {873, 5}, - {974, 2}, + {851, 1}, + {851, 2}, + {851, 2}, + {851, 2}, + {851, 4}, + {851, 5}, + {851, 6}, + {851, 4}, + {851, 5}, + {975, 2}, {1329, 1}, {1329, 3}, - {831, 3}, - {831, 3}, + {832, 3}, + {832, 3}, {729, 1}, {729, 3}, {729, 5}, - {794, 1}, - {794, 3}, - {982, 0}, - {982, 1}, + {795, 1}, + {795, 3}, + {983, 0}, + {983, 1}, {1200, 0}, {1200, 3}, - {856, 1}, - {856, 3}, + {858, 1}, + {858, 3}, {1166, 0}, {1166, 1}, {1165, 1}, {1165, 3}, - {983, 1}, - {983, 1}, + {984, 1}, + {984, 1}, {1167, 0}, {1167, 3}, - {883, 1}, - {883, 2}, + {855, 1}, + {855, 2}, {937, 0}, {937, 1}, {796, 1}, {796, 1}, {918, 1}, {918, 2}, - {1021, 0}, - {1021, 1}, + {1022, 0}, + {1022, 1}, {1181, 2}, {1181, 1}, {912, 2}, @@ -3802,10 +3802,10 @@ var ( {1313, 0}, {1313, 1}, {1313, 1}, - {984, 1}, - {984, 2}, - {985, 0}, {985, 1}, + {985, 2}, + {986, 0}, + {986, 1}, {1171, 7}, {1171, 7}, {1171, 7}, @@ -3817,86 +3817,86 @@ var ( {1224, 2}, {1225, 0}, {1225, 1}, - {893, 5}, - {1065, 3}, + {895, 5}, {1066, 3}, + {1067, 3}, {1231, 0}, {1231, 1}, {1231, 1}, {1231, 2}, {1231, 2}, - {1088, 1}, - {1088, 1}, - {1088, 2}, - {1088, 2}, - {1088, 2}, + {1089, 1}, + {1089, 1}, + {1089, 2}, + {1089, 2}, + {1089, 2}, {1178, 1}, {1178, 1}, {1178, 1}, - {1059, 1}, - {1059, 3}, - {1059, 4}, + {1060, 1}, + {1060, 3}, + {1060, 4}, {701, 4}, {701, 4}, + {1059, 1}, + {1059, 1}, + {1059, 1}, + {1059, 1}, {1058, 1}, {1058, 1}, {1058, 1}, - {1058, 1}, - {1057, 1}, - {1057, 1}, - {1057, 1}, - {1110, 1}, - {1110, 2}, - {1110, 2}, - {805, 1}, - {805, 1}, - {805, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {997, 12}, - {1013, 3}, - {993, 13}, + {1111, 1}, + {1111, 2}, + {1111, 2}, + {806, 1}, + {806, 1}, + {806, 1}, + {1117, 1}, + {1117, 1}, + {1117, 1}, + {998, 12}, + {1014, 3}, + {994, 13}, {1207, 0}, {1207, 3}, {822, 1}, {822, 3}, - {812, 3}, - {812, 4}, - {1042, 0}, - {1042, 1}, - {1042, 1}, - {1042, 2}, - {1042, 2}, + {813, 3}, + {813, 4}, + {1043, 0}, + {1043, 1}, + {1043, 1}, + {1043, 2}, + {1043, 2}, {1206, 0}, {1206, 1}, {1206, 1}, {1206, 1}, - {963, 4}, - {963, 3}, - {991, 5}, + {964, 4}, + {964, 3}, + {992, 5}, {801, 1}, - {865, 1}, - {832, 4}, - {832, 4}, - {832, 4}, - {832, 2}, - {832, 1}, + {868, 1}, + {833, 4}, + {833, 4}, + {833, 4}, + {833, 2}, + {833, 1}, {1175, 0}, {1175, 1}, {916, 1}, {916, 2}, {915, 12}, {915, 7}, - {1064, 0}, - {1064, 4}, - {1064, 4}, + {1065, 0}, + {1065, 4}, + {1065, 4}, {779, 0}, {779, 1}, - {1077, 0}, - {1077, 6}, - {1119, 6}, - {1119, 5}, + {1078, 0}, + {1078, 6}, + {1120, 6}, + {1120, 5}, {1247, 0}, {1247, 3}, {1248, 1}, @@ -3907,24 +3907,24 @@ var ( {1248, 4}, {1248, 3}, {1248, 1}, - {1051, 0}, - {1051, 1}, + {1052, 0}, + {1052, 1}, {1289, 0}, {1289, 4}, {1288, 0}, {1288, 2}, {1249, 0}, {1249, 2}, - {1076, 0}, + {1077, 0}, + {1077, 3}, + {1076, 1}, {1076, 3}, - {1075, 1}, - {1075, 3}, {933, 5}, {1287, 0}, {1287, 3}, {1286, 1}, {1286, 3}, - {1118, 3}, + {1119, 3}, {932, 0}, {932, 2}, {798, 3}, @@ -3945,9 +3945,9 @@ var ( {1246, 5}, {1246, 1}, {1246, 1}, - {1018, 0}, - {1018, 1}, - {1018, 1}, + {1019, 0}, + {1019, 1}, + {1019, 1}, {1152, 0}, {1152, 1}, {1173, 0}, @@ -3961,7 +3961,7 @@ var ( {1174, 1}, {1217, 2}, {1217, 4}, - {1000, 11}, + {1001, 11}, {1244, 0}, {1244, 2}, {1306, 0}, @@ -3981,128 +3981,129 @@ var ( {1307, 0}, {1307, 4}, {1307, 4}, - {1005, 2}, - {767, 13}, - {767, 9}, - {789, 10}, - {795, 1}, - {795, 1}, - {795, 2}, - {795, 2}, - {833, 1}, - {1007, 4}, - {1009, 7}, - {1015, 6}, + {1006, 2}, + {764, 13}, + {764, 9}, + {783, 10}, + {791, 1}, + {791, 1}, + {791, 2}, + {791, 2}, + {834, 1}, + {1008, 4}, + {1010, 7}, + {1016, 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}, + {1018, 4}, + {1018, 6}, + {1017, 3}, + {1017, 5}, + {1012, 3}, + {1012, 5}, + {1015, 3}, + {1015, 5}, + {1015, 4}, + {896, 0}, + {896, 1}, + {896, 1}, + {1125, 1}, + {1125, 1}, {723, 0}, {723, 1}, - {1019, 0}, + {1020, 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}, + {1129, 3}, + {1026, 1}, + {1026, 1}, + {1026, 1}, + {1025, 2}, + {1025, 3}, + {1025, 2}, + {1025, 4}, + {1025, 7}, + {1025, 5}, + {1025, 7}, + {1025, 5}, + {1025, 3}, {1182, 1}, {1182, 1}, {1182, 1}, {1182, 1}, {1182, 1}, {1182, 1}, - {975, 5}, - {975, 5}, - {976, 2}, - {976, 2}, - {976, 2}, + {976, 5}, + {976, 5}, + {977, 2}, + {977, 2}, + {977, 2}, {1177, 1}, {1177, 3}, - {879, 0}, - {879, 2}, - {876, 1}, - {876, 1}, - {875, 1}, - {875, 1}, - {875, 1}, - {875, 1}, - {875, 1}, - {875, 1}, - {875, 1}, - {875, 1}, - {880, 1}, - {880, 1}, + {883, 0}, + {883, 2}, {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}, + {879, 1}, + {879, 1}, + {879, 1}, + {879, 1}, + {879, 1}, + {879, 1}, + {879, 1}, + {879, 1}, + {884, 1}, + {884, 1}, + {884, 1}, + {884, 1}, + {881, 1}, + {881, 1}, + {881, 2}, + {882, 3}, + {882, 3}, + {882, 3}, + {882, 3}, + {882, 5}, + {882, 3}, + {882, 3}, + {882, 3}, + {882, 3}, + {882, 6}, + {882, 3}, + {882, 3}, + {882, 3}, + {882, 3}, + {882, 3}, + {882, 3}, {730, 1}, {748, 1}, {722, 1}, {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}, + {1072, 1}, + {1072, 1}, + {1072, 1}, + {1086, 3}, + {993, 8}, + {1118, 4}, + {1095, 4}, + {965, 6}, + {1009, 4}, + {1106, 5}, {1202, 0}, {1202, 2}, {1201, 0}, {1201, 3}, {1235, 0}, {1235, 1}, - {1022, 0}, - {1022, 1}, - {1022, 2}, - {1022, 2}, - {1022, 2}, - {1022, 2}, + {1023, 0}, + {1023, 1}, + {1023, 2}, + {1023, 2}, + {1023, 2}, + {1023, 2}, {1204, 0}, {1204, 3}, {1204, 3}, @@ -4129,13 +4130,13 @@ var ( {720, 1}, {761, 1}, {761, 3}, - {1056, 1}, - {1056, 3}, - {811, 0}, - {811, 1}, - {1032, 0}, + {1057, 1}, + {1057, 3}, + {812, 0}, + {812, 1}, + {1033, 0}, + {1033, 1}, {1032, 1}, - {1031, 1}, {718, 3}, {718, 3}, {718, 4}, @@ -4168,14 +4169,14 @@ var ( {717, 4}, {717, 3}, {717, 1}, - {1089, 1}, - {1089, 1}, + {1090, 1}, + {1090, 1}, {1215, 0}, {1215, 2}, - {1026, 1}, - {1026, 3}, - {1026, 5}, - {1026, 2}, + {1027, 1}, + {1027, 3}, + {1027, 5}, + {1027, 2}, {1186, 0}, {1186, 1}, {1185, 1}, @@ -4190,34 +4191,34 @@ var ( {1151, 0}, {1151, 1}, {908, 3}, - {763, 0}, - {763, 2}, + {765, 0}, + {765, 2}, {769, 0}, {769, 3}, - {838, 0}, - {838, 1}, - {857, 0}, - {857, 1}, + {839, 0}, + {839, 1}, {859, 0}, - {859, 2}, - {858, 3}, - {858, 1}, - {858, 3}, - {858, 2}, - {858, 1}, - {858, 1}, + {859, 1}, + {861, 0}, + {861, 2}, + {860, 3}, + {860, 1}, + {860, 3}, + {860, 2}, + {860, 1}, + {860, 1}, {926, 1}, {926, 3}, {926, 3}, {1208, 0}, {1208, 1}, - {841, 2}, - {841, 2}, - {887, 1}, - {887, 1}, - {887, 1}, - {839, 1}, - {839, 1}, + {842, 2}, + {842, 2}, + {890, 1}, + {890, 1}, + {890, 1}, + {840, 1}, + {840, 1}, {648, 1}, {648, 1}, {648, 1}, @@ -4675,43 +4676,43 @@ var ( {649, 1}, {649, 1}, {649, 1}, - {978, 2}, + {979, 2}, {1254, 1}, {1254, 3}, {1254, 4}, {1254, 6}, - {773, 9}, - {1044, 0}, + {770, 9}, + {1045, 0}, + {1045, 1}, + {1044, 5}, + {1044, 4}, + {1044, 4}, + {1044, 4}, + {1044, 4}, + {1044, 2}, {1044, 1}, - {1043, 5}, - {1043, 4}, - {1043, 4}, - {1043, 4}, - {1043, 4}, - {1043, 2}, - {1043, 1}, - {1043, 1}, - {1043, 1}, - {1043, 1}, - {1043, 2}, - {958, 1}, - {958, 1}, - {956, 1}, - {956, 3}, + {1044, 1}, + {1044, 1}, + {1044, 1}, + {1044, 2}, + {959, 1}, + {959, 1}, + {957, 1}, + {957, 3}, {826, 3}, {1305, 0}, {1305, 1}, {1304, 3}, {1304, 1}, - {790, 1}, - {790, 1}, - {986, 3}, + {784, 1}, + {784, 1}, + {987, 3}, {1168, 0}, {1168, 1}, {1168, 3}, {1232, 0}, {1232, 5}, - {775, 6}, + {772, 6}, {699, 1}, {699, 1}, {699, 1}, @@ -4728,12 +4729,12 @@ var ( {700, 2}, {1144, 1}, {1144, 3}, - {966, 2}, + {967, 2}, {753, 3}, - {881, 1}, - {881, 3}, - {851, 1}, - {851, 2}, + {885, 1}, + {885, 3}, + {852, 1}, + {852, 2}, {1243, 1}, {1243, 1}, {930, 0}, @@ -4792,8 +4793,8 @@ var ( {711, 3}, {768, 1}, {768, 1}, - {771, 1}, - {771, 1}, + {774, 1}, + {774, 1}, {802, 0}, {802, 1}, {917, 0}, @@ -4829,8 +4830,8 @@ var ( {705, 1}, {705, 1}, {705, 1}, - {1070, 0}, - {1070, 2}, + {1071, 0}, + {1071, 2}, {709, 1}, {709, 1}, {709, 1}, @@ -4937,48 +4938,48 @@ var ( {816, 1}, {816, 1}, {816, 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}, - {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}, + {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}, + {1183, 0}, + {1183, 1}, + {1314, 1}, + {1314, 2}, + {1136, 4}, + {1180, 0}, + {1180, 2}, + {980, 2}, + {980, 3}, + {980, 1}, + {980, 1}, + {980, 2}, + {980, 2}, + {980, 2}, + {980, 2}, + {980, 2}, + {980, 1}, + {980, 1}, + {980, 2}, + {980, 1}, {824, 1}, - {866, 0}, - {866, 1}, + {824, 1}, + {824, 1}, + {869, 0}, + {869, 1}, {724, 1}, {724, 3}, {782, 1}, @@ -4987,32 +4988,32 @@ var ( {902, 4}, {948, 1}, {948, 3}, - {890, 0}, - {890, 2}, - {1086, 0}, - {1086, 1}, - {1083, 4}, + {892, 0}, + {892, 2}, + {1087, 0}, + {1087, 1}, + {1084, 4}, {1253, 1}, {1253, 1}, - {1023, 2}, - {1023, 4}, + {1024, 2}, + {1024, 4}, {1302, 1}, {1302, 3}, - {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}, + {1003, 3}, + {1004, 1}, + {1004, 1}, + {870, 1}, + {870, 2}, + {988, 4}, + {988, 4}, + {988, 5}, + {988, 2}, + {988, 3}, + {988, 1}, + {988, 2}, + {1110, 1}, + {1094, 1}, + {1039, 2}, {739, 3}, {740, 3}, {741, 7}, @@ -5026,17 +5027,17 @@ var ( {1295, 0}, {1295, 1}, {1295, 1}, - {1092, 0}, - {1092, 4}, + {1093, 0}, + {1093, 4}, {738, 7}, {738, 6}, {738, 5}, {738, 6}, {738, 6}, - {750, 2}, - {750, 2}, - {752, 2}, - {752, 3}, + {749, 2}, + {749, 2}, + {751, 2}, + {751, 3}, {1141, 3}, {1141, 1}, {914, 4}, @@ -5062,19 +5063,19 @@ var ( {1319, 1}, {1318, 1}, {1318, 1}, - {960, 2}, - {960, 2}, - {960, 2}, - {960, 4}, - {960, 2}, + {961, 2}, + {961, 2}, + {961, 2}, + {961, 4}, + {961, 2}, {1317, 4}, {1138, 1}, {1138, 2}, {1138, 2}, {1138, 2}, {1138, 4}, - {749, 0}, - {749, 1}, + {752, 0}, + {752, 1}, {734, 2}, {1320, 1}, {1320, 1}, @@ -5089,29 +5090,29 @@ var ( {715, 6}, {715, 6}, {715, 9}, - {1072, 0}, - {1072, 3}, - {1072, 3}, {1073, 0}, - {1073, 2}, - {864, 0}, - {864, 2}, - {864, 2}, + {1073, 3}, + {1073, 3}, + {1074, 0}, + {1074, 2}, + {867, 0}, + {867, 2}, + {867, 2}, {1237, 0}, {1237, 2}, {1237, 2}, {1292, 1}, - {869, 1}, - {869, 3}, - {834, 1}, - {834, 4}, - {787, 1}, - {787, 1}, - {786, 6}, - {786, 2}, - {786, 3}, - {842, 0}, - {842, 4}, + {874, 1}, + {874, 3}, + {835, 1}, + {835, 4}, + {789, 1}, + {789, 1}, + {788, 6}, + {788, 2}, + {788, 3}, + {843, 0}, + {843, 4}, {901, 0}, {901, 1}, {900, 1}, @@ -5124,35 +5125,35 @@ var ( {1205, 3}, {1205, 3}, {924, 5}, - {840, 0}, - {840, 1}, - {840, 3}, - {840, 1}, - {840, 3}, - {1040, 1}, - {1040, 2}, - {1041, 0}, + {841, 0}, + {841, 1}, + {841, 3}, + {841, 1}, + {841, 3}, {1041, 1}, - {783, 3}, - {783, 5}, - {783, 7}, - {783, 7}, - {783, 9}, - {783, 4}, - {783, 6}, - {783, 3}, - {783, 5}, + {1041, 2}, + {1042, 0}, + {1042, 1}, + {785, 3}, + {785, 5}, + {785, 7}, + {785, 7}, + {785, 9}, + {785, 4}, + {785, 6}, + {785, 3}, + {785, 5}, {803, 1}, {803, 1}, - {1074, 0}, - {1074, 1}, - {809, 1}, - {809, 2}, - {809, 2}, - {1049, 0}, - {1049, 2}, - {861, 1}, - {861, 1}, + {1075, 0}, + {1075, 1}, + {810, 1}, + {810, 2}, + {810, 2}, + {1050, 0}, + {1050, 2}, + {863, 1}, + {863, 1}, {1260, 1}, {1260, 1}, {1189, 1}, @@ -5165,15 +5166,15 @@ var ( {754, 5}, {815, 0}, {815, 1}, - {1100, 1}, - {1100, 1}, - {1100, 1}, - {1100, 1}, - {1100, 1}, - {1100, 1}, - {1100, 1}, - {1100, 1}, - {1100, 1}, + {1101, 1}, + {1101, 1}, + {1101, 1}, + {1101, 1}, + {1101, 1}, + {1101, 1}, + {1101, 1}, + {1101, 1}, + {1101, 1}, {1262, 0}, {1262, 1}, {1263, 2}, @@ -5181,67 +5182,67 @@ var ( {847, 1}, {903, 0}, {903, 1}, - {1101, 1}, - {1101, 1}, + {1102, 1}, + {1102, 1}, {1261, 1}, {946, 0}, {946, 1}, - {868, 0}, - {868, 5}, + {872, 0}, + {872, 5}, {696, 3}, {696, 3}, {696, 3}, - {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}, - {751, 2}, - {746, 3}, - {746, 3}, - {745, 4}, - {745, 4}, - {745, 5}, - {745, 2}, - {745, 2}, + {871, 0}, + {871, 3}, + {871, 3}, + {871, 4}, + {871, 5}, + {871, 4}, + {871, 5}, + {871, 5}, + {871, 4}, + {1064, 0}, + {1064, 2}, + {750, 1}, + {750, 1}, + {750, 2}, + {750, 2}, + {745, 3}, {745, 3}, - {744, 1}, + {744, 4}, + {744, 4}, + {744, 5}, + {744, 2}, + {744, 2}, {744, 3}, + {743, 1}, + {743, 3}, {742, 1}, {742, 1}, {1265, 2}, {1265, 2}, {1265, 2}, {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}, + {981, 9}, + {981, 9}, + {873, 2}, + {873, 4}, + {873, 6}, + {873, 4}, + {873, 4}, + {873, 3}, + {873, 6}, + {873, 6}, + {1105, 3}, + {1104, 6}, + {1103, 1}, + {1103, 1}, + {1103, 1}, {1266, 3}, {1266, 1}, {1266, 1}, - {952, 1}, - {952, 3}, + {953, 1}, + {953, 3}, {905, 3}, {905, 2}, {905, 2}, @@ -5250,33 +5251,33 @@ var ( {1212, 2}, {1212, 2}, {1212, 1}, - {845, 1}, - {845, 1}, - {845, 1}, - {810, 1}, - {810, 1}, + {827, 1}, + {827, 1}, + {827, 1}, + {811, 1}, + {811, 1}, {817, 1}, {817, 3}, - {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}, + {887, 1}, + {887, 3}, + {887, 3}, + {960, 3}, + {960, 4}, + {960, 4}, + {960, 4}, + {960, 3}, + {960, 3}, + {960, 2}, + {960, 4}, + {960, 4}, + {960, 2}, + {960, 2}, {1160, 1}, {1160, 1}, - {793, 1}, - {793, 1}, - {852, 1}, - {852, 1}, + {794, 1}, + {794, 1}, + {853, 1}, + {853, 1}, {1135, 1}, {1135, 3}, {714, 1}, @@ -5289,75 +5290,75 @@ var ( {760, 2}, {848, 1}, {848, 3}, - {1078, 1}, - {1078, 4}, - {872, 1}, - {807, 1}, - {807, 1}, - {785, 3}, - {785, 2}, + {1079, 1}, + {1079, 4}, + {877, 1}, + {808, 1}, + {808, 1}, + {787, 3}, + {787, 2}, {944, 1}, {944, 1}, - {806, 1}, - {806, 1}, - {844, 1}, - {844, 3}, - {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}, + {807, 1}, + {807, 1}, + {845, 1}, + {845, 3}, + {962, 3}, + {962, 5}, + {962, 6}, + {962, 4}, + {962, 4}, + {962, 5}, + {962, 5}, + {962, 5}, + {962, 6}, + {962, 4}, + {962, 5}, + {962, 6}, + {962, 4}, + {962, 3}, + {962, 3}, + {962, 4}, + {962, 4}, + {962, 5}, + {962, 5}, + {962, 3}, + {962, 3}, + {962, 3}, + {962, 3}, + {962, 3}, + {962, 3}, + {962, 3}, + {962, 3}, {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}, + {1037, 5}, + {1061, 1}, + {1061, 3}, + {1108, 3}, + {1108, 4}, + {1108, 4}, + {1108, 5}, + {1108, 4}, + {1108, 5}, + {1108, 4}, + {1108, 4}, + {1108, 6}, + {1108, 4}, + {1108, 8}, + {1108, 2}, + {1108, 5}, + {1108, 3}, + {1108, 3}, + {1108, 2}, + {1108, 5}, + {1108, 2}, + {1108, 2}, + {1108, 4}, {1269, 2}, {1269, 2}, {1269, 4}, @@ -5365,15 +5366,15 @@ var ( {1272, 1}, {1271, 1}, {1271, 3}, - {1106, 1}, - {1106, 1}, - {1106, 2}, - {1106, 2}, - {1106, 2}, - {1106, 1}, - {1106, 1}, - {1106, 1}, - {1106, 1}, + {1107, 1}, + {1107, 1}, + {1107, 2}, + {1107, 2}, + {1107, 2}, + {1107, 1}, + {1107, 1}, + {1107, 1}, + {1107, 1}, {1270, 0}, {1270, 3}, {1303, 0}, @@ -5381,8 +5382,8 @@ var ( {1267, 1}, {1267, 1}, {1267, 1}, - {791, 1}, - {791, 1}, + {792, 1}, + {792, 1}, {1273, 1}, {1273, 1}, {1273, 1}, @@ -5426,12 +5427,12 @@ var ( {922, 0}, {922, 1}, {922, 1}, - {1068, 0}, - {1068, 1}, - {827, 0}, - {827, 2}, - {1108, 2}, - {1030, 3}, + {1069, 0}, + {1069, 1}, + {828, 0}, + {828, 2}, + {1109, 2}, + {1031, 3}, {936, 1}, {936, 3}, {1193, 1}, @@ -5450,170 +5451,170 @@ var ( {823, 0}, {823, 1}, {823, 1}, - {1123, 0}, - {1123, 1}, + {1124, 0}, + {1124, 1}, {950, 0}, {950, 2}, {1322, 0}, {1322, 3}, - {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}, - {835, 1}, - {835, 1}, - {835, 1}, - {835, 1}, - {835, 1}, - {835, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {1114, 1}, + {952, 1}, + {952, 1}, + {952, 1}, + {952, 1}, + {952, 1}, + {952, 1}, + {952, 1}, + {952, 1}, + {952, 1}, + {952, 1}, + {952, 1}, + {952, 1}, + {952, 1}, + {836, 1}, + {836, 1}, + {836, 1}, + {836, 1}, + {836, 1}, + {836, 1}, + {836, 1}, + {836, 1}, + {836, 1}, {1280, 1}, {1280, 3}, - {885, 2}, - {981, 1}, - {981, 1}, + {888, 2}, + {982, 1}, + {982, 1}, {949, 1}, {949, 1}, - {1121, 1}, - {1121, 3}, + {1122, 1}, + {1122, 3}, {1290, 0}, {1290, 3}, - {828, 1}, - {828, 4}, - {828, 4}, - {828, 4}, - {828, 3}, - {828, 4}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 1}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 3}, - {828, 2}, - {828, 2}, - {828, 3}, - {828, 3}, - {828, 5}, - {828, 3}, + {829, 1}, + {829, 4}, + {829, 4}, + {829, 4}, + {829, 3}, + {829, 4}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 1}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 3}, + {829, 2}, + {829, 2}, + {829, 3}, + {829, 3}, + {829, 5}, + {829, 3}, {821, 0}, {821, 1}, - {1115, 1}, - {1115, 1}, - {998, 0}, - {998, 1}, + {1116, 1}, + {1116, 1}, + {999, 0}, + {999, 1}, {904, 1}, {904, 2}, {904, 3}, @@ -5655,9 +5656,9 @@ var ( {1211, 1}, {1158, 1}, {1158, 1}, - {1069, 0}, - {1069, 1}, - {1069, 1}, + {1070, 0}, + {1070, 1}, + {1070, 1}, {1190, 1}, {1190, 1}, {1190, 1}, @@ -5681,8 +5682,8 @@ var ( {1285, 1}, {1285, 3}, {1285, 2}, - {882, 1}, - {882, 1}, + {886, 1}, + {886, 1}, {1228, 1}, {1228, 2}, {1228, 2}, @@ -5709,79 +5710,79 @@ var ( {1297, 2}, {1297, 1}, {1297, 1}, - {863, 1}, - {863, 1}, - {863, 1}, - {863, 1}, + {866, 1}, + {866, 1}, + {866, 1}, + {866, 1}, {1176, 1}, {1176, 2}, {1176, 2}, {1176, 2}, {1176, 3}, {747, 3}, - {770, 0}, - {770, 1}, - {854, 1}, - {854, 1}, - {854, 1}, - {855, 0}, - {855, 2}, - {886, 0}, - {886, 1}, - {886, 1}, - {892, 5}, + {771, 0}, + {771, 1}, + {856, 1}, + {856, 1}, + {856, 1}, + {857, 0}, + {857, 2}, + {889, 0}, + {889, 1}, + {889, 1}, + {894, 5}, {1233, 0}, {1233, 1}, - {784, 0}, - {784, 2}, - {784, 3}, + {786, 0}, + {786, 2}, + {786, 3}, {1234, 0}, {1234, 2}, {759, 2}, {759, 1}, {759, 2}, - {1067, 0}, - {1067, 2}, + {1068, 0}, + {1068, 2}, {1283, 1}, {1283, 3}, {951, 1}, {951, 1}, {951, 1}, - {1127, 1}, - {1127, 3}, + {1128, 1}, + {1128, 3}, {725, 1}, {725, 1}, {1284, 1}, {1284, 1}, {1284, 1}, - {776, 1}, - {776, 2}, - {766, 10}, - {766, 8}, + {773, 1}, + {773, 2}, + {763, 10}, + {763, 8}, {1133, 2}, {777, 2}, {778, 0}, {778, 1}, {1330, 0}, {1330, 1}, - {999, 7}, - {995, 4}, - {971, 7}, - {971, 9}, - {965, 3}, + {1000, 7}, + {996, 4}, + {972, 7}, + {972, 9}, + {966, 3}, {1210, 2}, {1210, 6}, - {870, 2}, + {875, 2}, {906, 1}, {906, 3}, - {989, 0}, - {989, 2}, + {990, 0}, + {990, 2}, {1170, 1}, {1170, 2}, - {988, 2}, - {988, 2}, - {988, 2}, - {988, 2}, + {989, 2}, + {989, 2}, + {989, 2}, + {989, 2}, {942, 0}, {942, 1}, {941, 2}, @@ -5795,10 +5796,10 @@ var ( {943, 2}, {943, 2}, {943, 2}, - {1080, 0}, + {1081, 0}, + {1081, 1}, {1080, 1}, - {1079, 1}, - {1079, 2}, + {1080, 2}, {935, 2}, {935, 2}, {935, 1}, @@ -5814,25 +5815,25 @@ var ( {1153, 5}, {1153, 4}, {1154, 1}, - {1037, 1}, - {1037, 1}, - {1099, 1}, + {1038, 1}, + {1038, 1}, + {1100, 1}, {1259, 1}, {1259, 3}, - {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}, + {878, 1}, + {878, 1}, + {878, 1}, + {878, 1}, + {878, 1}, + {878, 1}, + {878, 1}, + {878, 1}, + {991, 7}, + {1007, 5}, + {1007, 7}, + {1036, 9}, + {1034, 7}, + {1035, 4}, {1140, 0}, {1140, 3}, {1140, 3}, @@ -5846,8 +5847,8 @@ var ( {945, 1}, {945, 3}, {945, 3}, - {1098, 1}, - {1098, 3}, + {1099, 1}, + {1099, 3}, {938, 1}, {938, 4}, {939, 1}, @@ -5885,37 +5886,37 @@ var ( {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}, + {1062, 0}, + {1062, 1}, + {1062, 1}, + {1062, 1}, + {1085, 1}, + {1085, 3}, + {1085, 3}, + {1085, 3}, + {1085, 1}, + {1098, 7}, + {1097, 4}, + {864, 15}, {1203, 0}, {1203, 3}, {1161, 0}, {1161, 3}, - {1054, 0}, - {1054, 1}, - {1028, 0}, - {1028, 2}, + {1055, 0}, + {1055, 1}, + {1029, 0}, + {1029, 2}, {820, 1}, {820, 1}, {1187, 2}, {1187, 1}, - {1027, 3}, - {1027, 4}, - {1027, 3}, - {1027, 3}, - {836, 1}, - {836, 1}, - {836, 1}, + {1028, 3}, + {1028, 4}, + {1028, 3}, + {1028, 3}, + {837, 1}, + {837, 1}, + {837, 1}, {928, 0}, {928, 3}, {1278, 0}, @@ -5926,60 +5927,60 @@ var ( {1220, 2}, {1219, 3}, {1219, 1}, - {1052, 3}, + {1053, 3}, {1131, 2}, - {1055, 3}, - {1125, 1}, - {1125, 1}, - {1122, 2}, + {1056, 3}, + {1126, 1}, + {1126, 1}, + {1123, 2}, {1222, 1}, {1222, 2}, {1222, 1}, {1222, 2}, {1291, 1}, {1291, 3}, + {1049, 2}, + {1049, 3}, + {1049, 3}, + {1048, 1}, {1048, 2}, - {1048, 3}, - {1048, 3}, - {1047, 1}, - {1047, 2}, - {1053, 3}, - {1010, 5}, - {994, 6}, - {967, 6}, - {996, 6}, + {1054, 3}, + {1011, 5}, + {995, 7}, + {968, 6}, + {997, 6}, {1172, 0}, {1172, 1}, {1264, 1}, {1264, 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}, + {898, 3}, + {898, 3}, + {898, 3}, + {898, 3}, + {898, 3}, + {898, 1}, + {898, 2}, + {898, 3}, + {898, 1}, + {898, 2}, + {898, 3}, + {898, 1}, + {898, 2}, + {898, 1}, + {898, 1}, + {898, 2}, {799, 1}, {799, 2}, {799, 2}, - {1012, 4}, - {969, 5}, + {1013, 4}, + {970, 5}, {1145, 1}, {1145, 2}, - {968, 1}, - {968, 1}, - {968, 3}, - {968, 3}, - {1039, 8}, + {969, 1}, + {969, 1}, + {969, 3}, + {969, 3}, + {1040, 8}, {1227, 0}, {1227, 2}, {1226, 0}, @@ -5988,92 +5989,92 @@ var ( {1251, 2}, {1250, 0}, {1250, 2}, - {1020, 1}, - {957, 1}, - {957, 3}, - {896, 2}, - {1082, 5}, - {1082, 6}, - {1082, 9}, - {1082, 10}, - {1082, 4}, + {1021, 1}, + {958, 1}, + {958, 3}, + {897, 2}, + {1083, 5}, + {1083, 6}, + {1083, 9}, + {1083, 10}, + {1083, 4}, } yyXErrors = map[yyXError]string{} - yyParseTab = [4155][]uint16{ + yyParseTab = [4157][]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, 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}, - {158: 6523, 553: 6522}, + {1986, 1986, 59: 2478, 80: 2593, 82: 2459, 91: 2489, 152: 2461, 157: 2487, 161: 2458, 165: 2483, 196: 2508, 203: 2605, 206: 2454, 215: 2507, 2474, 2460, 232: 2486, 237: 2464, 240: 2484, 242: 2455, 244: 2490, 261: 2476, 265: 2475, 272: 2488, 274: 2456, 277: 2477, 288: 2469, 459: 2498, 2497, 483: 2601, 2496, 491: 2482, 498: 2506, 511: 2596, 515: 2472, 553: 2495, 555: 2481, 632: 2491, 635: 2604, 640: 2457, 2595, 653: 2452, 656: 2463, 661: 2462, 666: 2505, 673: 2453, 696: 2502, 731: 2465, 738: 2504, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 2575, 2574, 2468, 763: 2594, 2466, 770: 2558, 772: 2569, 2585, 783: 2467, 791: 2524, 804: 2599, 809: 2512, 838: 2597, 846: 2479, 851: 2519, 855: 2522, 864: 2561, 870: 2566, 873: 2576, 915: 2531, 919: 2470, 955: 2600, 962: 2510, 964: 2511, 2514, 2515, 968: 2517, 970: 2516, 972: 2513, 974: 2518, 2520, 2521, 978: 2480, 2557, 981: 2527, 991: 2535, 2528, 2529, 2530, 2536, 2534, 2537, 2538, 1000: 2533, 2532, 1003: 2523, 2485, 2471, 2539, 2551, 2540, 2541, 2542, 2544, 2548, 2545, 2549, 2550, 2543, 2547, 2546, 1020: 2509, 1024: 2525, 2526, 2473, 1030: 2553, 2552, 1034: 2555, 2556, 2554, 1039: 2591, 2559, 1047: 2603, 2602, 2560, 1054: 2562, 1056: 2588, 1083: 2563, 2564, 1086: 2565, 1088: 2570, 1091: 2567, 2568, 1094: 2590, 2571, 2598, 2573, 2572, 1104: 2578, 2577, 2581, 1108: 2582, 1110: 2589, 1113: 2579, 2592, 1118: 2580, 1129: 2583, 2584, 2587, 1133: 2586, 1277: 2450, 1280: 2451}, + {2449}, + {2448, 6604}, + {16: 6545, 132: 6542, 160: 6543, 185: 6546, 331: 6544, 474: 4072, 553: 1803, 568: 5904, 834: 6541, 839: 4071}, + {160: 6526, 553: 6525}, // 5 - {553: 6516}, - {553: 6511}, - {362: 6492, 475: 6493, 553: 2301, 1275: 6491}, - {329: 6447, 553: 6446}, - {2269, 2269, 349: 6445, 356: 6444}, + {553: 6519}, + {553: 6514}, + {362: 6495, 475: 6496, 553: 2302, 1275: 6494}, + {329: 6450, 553: 6449}, + {2270, 2270, 349: 6448, 356: 6447}, // 10 - {386: 6433}, - {461: 6432}, - {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}, + {386: 6436}, + {461: 6435}, + {2237, 2237, 81: 5747, 492: 5745, 844: 5746, 988: 6434}, + {16: 2036, 92: 2036, 99: 2036, 132: 6249, 139: 2036, 151: 574, 158: 5402, 160: 6250, 162: 6171, 166: 6251, 185: 6253, 209: 5873, 6241, 494: 6248, 553: 2005, 568: 5904, 629: 6243, 635: 2130, 655: 2036, 663: 6245, 834: 6246, 922: 6252, 931: 5401, 1206: 6242, 1244: 6247, 1274: 6244}, + {16: 6178, 99: 6172, 110: 2005, 132: 6176, 151: 574, 158: 5402, 160: 6173, 162: 6171, 165: 997, 6174, 185: 6179, 209: 5873, 6167, 275: 6175, 553: 2005, 568: 5904, 635: 6169, 834: 6168, 922: 6177, 931: 6170}, // 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, 903: 6129}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 761: 6166}, + {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, 746: 5214, 847: 5215, 903: 6132}, + {2013, 2013}, {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, 789: 6127}, + {459: 2498, 484: 2496, 553: 2495, 632: 2491, 641: 2595, 696: 3773, 731: 2465, 738: 3772, 2492, 2493, 2494, 2503, 2501, 3774, 3775, 763: 6131, 6129, 783: 6130}, // 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, 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, 773: 2615, 775: 2616, 2614, 789: 2466, 795: 2613, 808: 2619, 835: 6081}, + {82: 2459, 152: 2461, 157: 2487, 161: 2458, 203: 6109, 324: 6108, 459: 2498, 2497, 484: 2496, 491: 2482, 498: 6112, 553: 2495, 555: 2481, 632: 2491, 641: 2595, 696: 6110, 731: 2465, 738: 6111, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 6118, 6117, 2468, 763: 2594, 2466, 770: 6115, 772: 6116, 6114, 783: 2467, 791: 6113, 804: 6124, 851: 6120, 855: 6121, 864: 6119, 870: 6122, 873: 6123, 952: 6107}, + {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, 491: 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, 491: 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, 491: 1980, 553: 1980, 555: 1980, 632: 1980, 640: 1980, 1980, 653: 1980, 731: 1980}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 6084, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 2498, 2497, 479: 6083, 484: 2496, 491: 2482, 553: 2495, 555: 2481, 632: 2491, 640: 6085, 2595, 648: 3806, 2665, 2666, 2664, 653: 2611, 696: 2612, 724: 6081, 731: 2465, 738: 2613, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 2619, 2618, 2468, 763: 2594, 2466, 770: 2616, 772: 2617, 2615, 783: 2467, 791: 2614, 809: 2620, 836: 6082}, // 25 - {553: 5998, 568: 5903, 833: 5997, 976: 6076}, - {553: 5998, 568: 5903, 833: 5997, 976: 5996}, - {132: 5994}, - {132: 5989}, - {132: 5983}, + {553: 5999, 568: 5904, 834: 5998, 977: 6077}, + {553: 5999, 568: 5904, 834: 5998, 977: 5997}, + {132: 5995}, + {132: 5990}, + {132: 5984}, // 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, 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, 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}, + {13: 3721, 16: 5839, 39: 5865, 5864, 98: 571, 107: 571, 110: 571, 125: 574, 132: 5828, 138: 574, 162: 5872, 180: 5837, 189: 574, 197: 5874, 5851, 204: 5860, 571, 209: 5873, 238: 5857, 260: 5856, 294: 5869, 299: 5838, 306: 5853, 5867, 309: 5845, 316: 5843, 318: 5859, 322: 5849, 325: 5858, 5832, 328: 5871, 330: 5841, 340: 5833, 348: 5847, 358: 5836, 5835, 366: 5870, 371: 5866, 5863, 5862, 387: 5854, 391: 5850, 486: 3722, 553: 5831, 634: 3720, 5840, 640: 5868, 661: 5830, 759: 5846, 899: 5861, 922: 5852, 927: 5842, 940: 5855, 1002: 5844, 1069: 5834, 1267: 5848, 1273: 5829}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 5817, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5819, 2665, 2666, 2664, 1254: 5818}, + {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, 746: 5214, 847: 5215, 903: 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, 481: 1020, 735: 5219, 5218, 5217, 824: 5220, 869: 5770}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5765, 2665, 2666, 2664}, // 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, 987: 5743}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5759, 2665, 2666, 2664}, + {165: 5757}, + {165: 998}, + {996, 996, 81: 5747, 492: 5745, 844: 5746, 988: 5744}, {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, 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}, + {461: 5743}, + {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, 5714, 5720, 5721, 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, 491: 823, 493: 823, 500: 5717, 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: 3433, 732: 3431, 3432, 735: 5219, 5218, 5217, 746: 5214, 755: 5713, 5716, 5712, 768: 5635, 774: 5710, 824: 5711, 847: 5709, 1101: 5719, 5715, 1262: 5708, 5718}, + {237, 237, 57: 237, 458: 237, 460: 237, 466: 237, 468: 237, 476: 237, 237, 479: 237, 237, 237, 483: 237, 487: 5683, 237, 2625, 237, 499: 237, 777: 2626, 5684, 1194: 5682}, + {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: 5673, 923: 5675, 946: 5674}, // 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, 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}, + {1258, 1258, 57: 1258, 458: 1258, 460: 1258, 466: 1258, 468: 1258, 476: 1258, 1258, 479: 1258, 1258, 1258, 483: 1258, 488: 1258, 490: 2628, 753: 2629, 797: 5669}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5664}, + {561: 3781, 897: 3780, 958: 3779}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5651, 2665, 2666, 2664, 914: 5650, 1141: 5648, 1255: 5649}, + {459: 2498, 2497, 484: 2496, 553: 2495, 632: 2491, 696: 5647, 738: 3766, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 3768, 3767, 3765}, // 50 {795, 795, 57: 795, 458: 795, 460: 795, 468: 795}, {794, 794, 57: 794, 458: 794, 460: 794, 468: 794}, - {466: 5631, 476: 5632, 5633, 1265: 5630}, - {473, 473, 466: 780, 476: 780, 780, 480: 2630, 488: 2631, 490: 2627, 753: 3775, 3776}, + {466: 5632, 476: 5633, 5634, 1265: 5631}, + {473, 473, 466: 780, 476: 780, 780, 480: 2631, 488: 2632, 490: 2628, 753: 3776, 3777}, {466: 783, 476: 783, 783}, // 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, 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}, + {238: 5616, 260: 5615}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 5499, 5504, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 5502, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 5501, 3253, 2739, 2743, 5505, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 5506, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 5500, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 5507, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 5503, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 464: 5509, 486: 3722, 554: 5513, 574: 5512, 634: 3720, 648: 5510, 2665, 2666, 2664, 759: 5514, 817: 5511, 960: 5515, 1135: 5508}, + {27: 5384, 196: 5389, 204: 5387, 206: 5382, 5388, 264: 5386, 300: 5385, 5390, 304: 5383, 319: 5391, 365: 5392, 571: 5381, 846: 5380}, + {31: 550, 110: 550, 125: 550, 136: 4622, 142: 550, 180: 550, 186: 550, 195: 550, 212: 550, 223: 550, 243: 550, 246: 550, 529: 550, 553: 550, 805: 4621, 823: 5353}, // 60 {541, 541}, {540, 540}, @@ -6175,30 +6176,30 @@ var ( {458, 458}, {457, 457}, {434, 434}, - {2: 380, 380, 380, 380, 380, 8: 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 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}, + {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: 5350, 1240: 5351}, // 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, 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, 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}, + {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, 746: 5214, 847: 5215, 903: 5216}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5212, 2665, 2666, 2664, 801: 5213}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 5057, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 5059, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 5065, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 5061, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 5058, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 5066, 3096, 2832, 3051, 5060, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 5063, 5167, 2746, 2982, 5064, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 5062, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 5068, 483: 5091, 555: 5085, 631: 5089, 5074, 635: 5084, 638: 5078, 641: 5087, 648: 3378, 2665, 2666, 2664, 653: 5079, 656: 5083, 661: 5080, 725: 5067, 731: 5082, 787: 5069, 804: 5073, 838: 5088, 846: 5086, 920: 5070, 938: 5071, 5077, 944: 5072, 5075, 954: 5081, 956: 5090, 1099: 5168}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 5057, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 5059, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 5065, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 5061, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 5058, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 5066, 3096, 2832, 3051, 5060, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 5063, 2745, 2746, 2982, 5064, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 5062, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 5068, 483: 5091, 555: 5085, 631: 5089, 5074, 635: 5084, 638: 5078, 641: 5087, 648: 3378, 2665, 2666, 2664, 653: 5079, 656: 5083, 661: 5080, 725: 5067, 731: 5082, 787: 5069, 804: 5073, 838: 5088, 846: 5086, 920: 5070, 938: 5071, 5077, 944: 5072, 5075, 954: 5081, 956: 5090, 1099: 5076}, // 150 - {32: 5015, 275: 5016}, - {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}, + {32: 5016, 275: 5017}, + {110: 5003, 553: 5004, 1126: 5015}, + {110: 5003, 553: 5004, 1126: 5002}, + {37: 4998, 143: 4999, 493: 2639, 722: 4997}, + {37: 56, 143: 56, 212: 4996, 493: 56}, // 155 - {290: 4978}, - {363: 2605}, - {315: 2606, 814: 2607}, - {919: 2609}, - {461: 2608}, + {290: 4979}, + {363: 2606}, + {315: 2607, 804: 2608}, + {919: 2610}, + {461: 2609}, // 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, 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}, + {186: 2623, 459: 2498, 2497, 484: 2496, 491: 2482, 553: 2495, 555: 2481, 632: 2491, 640: 2622, 2595, 653: 2611, 696: 2612, 731: 2465, 738: 2613, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 2619, 2618, 2468, 763: 2594, 2466, 770: 2616, 772: 2617, 2615, 783: 2467, 791: 2614, 809: 2620, 836: 2621}, + {474: 4072, 553: 1803, 839: 4071}, + {436, 436, 466: 780, 476: 780, 780, 480: 2631, 488: 2632, 490: 2628, 753: 3776, 3777}, {438, 438, 466: 781, 476: 781, 781}, // 165 {443, 443}, @@ -6210,56 +6211,56 @@ 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, 773: 2615, 775: 2616, 2614, 789: 2466, 795: 2613, 808: 2619, 835: 4064}, - {143: 2623}, + {186: 4066, 459: 2498, 2497, 484: 2496, 491: 2482, 553: 2495, 555: 2481, 632: 2491, 641: 2595, 653: 2611, 696: 2612, 731: 2465, 738: 2613, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 2619, 2618, 2468, 763: 2594, 2466, 770: 2616, 772: 2617, 2615, 783: 2467, 791: 2614, 809: 2620, 836: 4065}, + {143: 2624}, // 175 - {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}, + {237, 237, 480: 237, 488: 237, 2625, 237, 777: 2626, 2627}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 4064}, {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}, - {646: 2652}, + {1258, 1258, 480: 1258, 488: 1258, 490: 2628, 753: 2629, 797: 2630}, + {646: 2653}, // 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, 861: 2647}, - {8: 2634, 254: 2635, 1189: 2636}, + {834, 834, 480: 2631, 488: 2632, 754: 2633, 815: 2634}, + {493: 2639, 564: 2641, 722: 2638, 730: 2640, 863: 2648}, + {8: 2635, 255: 2636, 1189: 2637}, {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, 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}, + {493: 2639, 510: 840, 561: 840, 564: 2641, 722: 2638, 730: 2640, 863: 2642, 1184: 2643}, + {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, 164: 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, 491: 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}, - {846, 846, 7: 846, 57: 846, 162: 846, 458: 846, 460: 846, 466: 846, 468: 846, 476: 846, 846, 479: 846, 481: 846, 483: 846, 510: 846, 561: 846}, - {845, 845, 7: 845, 57: 845, 162: 845, 458: 845, 460: 845, 466: 845, 468: 845, 476: 845, 845, 479: 845, 481: 845, 483: 845, 510: 845, 561: 845}, + {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, 164: 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, 491: 1920, 510: 1920, 553: 1920, 561: 1920, 632: 1920, 634: 1920, 1920, 640: 1920, 644: 1920, 1920}, + {846, 846, 7: 846, 57: 846, 164: 846, 458: 846, 460: 846, 466: 846, 468: 846, 476: 846, 846, 479: 846, 481: 846, 483: 846, 510: 846, 561: 846}, + {845, 845, 7: 845, 57: 845, 164: 845, 458: 845, 460: 845, 466: 845, 468: 845, 476: 845, 845, 479: 845, 481: 845, 483: 845, 510: 845, 561: 845}, {510: 839, 561: 839}, - {510: 2644, 561: 2643, 1260: 2645}, + {510: 2645, 561: 2644, 1260: 2646}, // 195 {149: 844}, {149: 843}, - {149: 2646}, + {149: 2647}, {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}, + {838, 838, 7: 2649, 57: 838, 164: 2650, 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, 861: 2651}, - {493: 2638, 564: 2640, 722: 2637, 730: 2639, 861: 2650}, + {493: 2639, 564: 2641, 722: 2638, 730: 2640, 863: 2652}, + {493: 2639, 564: 2641, 722: 2638, 730: 2640, 863: 2651}, {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, 881: 3128}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 2657, 852: 3130, 885: 3129}, // 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}, - {715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 460: 715, 715, 715, 715, 465: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 476: 715, 715, 479: 715, 715, 715, 715, 715, 485: 715, 487: 715, 715, 715, 715, 715, 494: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 530: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 570: 715, 647: 4055}, - {1265, 1265, 7: 1265, 57: 1265, 124: 1265, 458: 1265, 460: 1265, 466: 1265, 468: 1265, 476: 1265, 1265, 479: 1265, 1265, 1265, 483: 1265, 488: 1265, 490: 1265, 3234, 494: 3232, 3233, 3231, 3229, 501: 1265, 1265, 510: 1265, 513: 1265, 1265, 4054, 4053, 720: 3230, 3228, 1243: 4052}, - {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}, + {1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 4061, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 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, 492: 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, 4058, 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, 492: 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}, + {715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 460: 715, 715, 715, 715, 465: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 476: 715, 715, 479: 715, 715, 715, 715, 715, 485: 715, 487: 715, 715, 715, 715, 492: 715, 494: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 530: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 570: 715, 647: 4056}, + {1265, 1265, 7: 1265, 57: 1265, 124: 1265, 458: 1265, 460: 1265, 466: 1265, 468: 1265, 476: 1265, 1265, 479: 1265, 1265, 1265, 483: 1265, 488: 1265, 490: 1265, 492: 3235, 494: 3233, 3234, 3232, 3230, 501: 1265, 1265, 510: 1265, 513: 1265, 1265, 4055, 4054, 720: 3231, 3229, 1243: 4053}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 4052}, // 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, 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, 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}, + {459: 4024}, + {1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 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, 4007, 1882, 485: 1882, 487: 1882, 1882, 1882, 1882, 492: 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, 4004, 4002, 4001, 4009, 4003, 4005, 4006, 4008, 1169: 4000, 1213: 3999}, + {1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 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, 492: 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, 3971, 3970, 465: 1830, 1830, 468: 1830, 1830, 1830, 3574, 3573, 3579, 1830, 476: 1830, 1830, 479: 1830, 1830, 1830, 1830, 1830, 485: 1830, 487: 1830, 1830, 1830, 1830, 492: 1830, 494: 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 3975, 1830, 3575, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 530: 3974, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 1830, 3972, 3576, 3577, 3570, 3580, 3569, 3578, 3571, 3572, 3981, 3982, 796: 3973, 1090: 3976, 1155: 3978, 1209: 3977, 1216: 3979, 1256: 3980}, + {1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 3967, 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, 492: 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}, {1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 652: 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777}, @@ -6267,7 +6268,7 @@ var ( {1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 652: 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775}, {1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 652: 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774}, // 220 - {1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1197, 1773, 1773, 1773, 1773, 465: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 476: 1773, 1773, 479: 1773, 1773, 1773, 1773, 1773, 485: 1773, 487: 1773, 1773, 1773, 1773, 1773, 494: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 530: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 570: 1773, 639: 1773, 642: 1773, 1773}, + {1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1197, 1773, 1773, 1773, 1773, 465: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 476: 1773, 1773, 479: 1773, 1773, 1773, 1773, 1773, 485: 1773, 487: 1773, 1773, 1773, 1773, 492: 1773, 494: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 530: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 570: 1773, 639: 1773, 642: 1773, 1773}, {1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 652: 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772}, {1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 652: 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771}, {1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 652: 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770}, @@ -6279,7 +6280,7 @@ var ( {1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 652: 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765}, {1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 652: 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764}, // 230 - {1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 3961, 1763, 1763, 1763, 1763, 465: 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 476: 1763, 1763, 479: 1763, 1763, 1763, 1763, 1763, 485: 1763, 487: 1763, 1763, 1763, 1763, 1763, 494: 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 530: 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 570: 1763, 639: 1763, 642: 1763, 1763}, + {1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 3962, 1763, 1763, 1763, 1763, 465: 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 476: 1763, 1763, 479: 1763, 1763, 1763, 1763, 1763, 485: 1763, 487: 1763, 1763, 1763, 1763, 492: 1763, 494: 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 530: 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 570: 1763, 639: 1763, 642: 1763, 1763}, {1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 652: 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762}, {1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 652: 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761}, {1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 652: 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760}, @@ -6292,7 +6293,7 @@ var ( {1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 652: 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754}, // 240 {1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 652: 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753}, - {1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1196, 1752, 1752, 1752, 1752, 465: 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 476: 1752, 1752, 479: 1752, 1752, 1752, 1752, 1752, 485: 1752, 487: 1752, 1752, 1752, 1752, 1752, 494: 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 530: 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 570: 1752, 639: 1752, 642: 1752, 1752}, + {1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1196, 1752, 1752, 1752, 1752, 465: 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 476: 1752, 1752, 479: 1752, 1752, 1752, 1752, 1752, 485: 1752, 487: 1752, 1752, 1752, 1752, 492: 1752, 494: 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 530: 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 570: 1752, 639: 1752, 642: 1752, 1752}, {1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 652: 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751}, {1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 652: 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750}, {1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 652: 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749}, @@ -6305,9 +6306,9 @@ var ( // 250 {1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 652: 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743}, {1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 652: 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742}, - {1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1193, 1741, 3960, 1741, 1741, 465: 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 476: 1741, 1741, 479: 1741, 1741, 1741, 1741, 1741, 485: 1741, 487: 1741, 1741, 1741, 1741, 1741, 494: 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 530: 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 570: 1741, 639: 1741, 642: 1741, 1741}, + {1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1193, 1741, 3961, 1741, 1741, 465: 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 476: 1741, 1741, 479: 1741, 1741, 1741, 1741, 1741, 485: 1741, 487: 1741, 1741, 1741, 1741, 492: 1741, 494: 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 530: 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 570: 1741, 639: 1741, 642: 1741, 1741}, {1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 652: 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740}, - {1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1191, 1739, 1739, 1739, 1739, 465: 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 476: 1739, 1739, 479: 1739, 1739, 1739, 1739, 1739, 485: 1739, 487: 1739, 1739, 1739, 1739, 1739, 494: 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 530: 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 570: 1739, 639: 1739, 642: 1739, 1739}, + {1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1191, 1739, 1739, 1739, 1739, 465: 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 476: 1739, 1739, 479: 1739, 1739, 1739, 1739, 1739, 485: 1739, 487: 1739, 1739, 1739, 1739, 492: 1739, 494: 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 530: 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 570: 1739, 639: 1739, 642: 1739, 1739}, // 255 {1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 652: 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738}, {1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 652: 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737}, @@ -6335,14 +6336,14 @@ var ( // 275 {1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 652: 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718}, {1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 652: 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717}, - {1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1187, 1716, 1716, 1716, 1716, 465: 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 476: 1716, 1716, 479: 1716, 1716, 1716, 1716, 1716, 485: 1716, 487: 1716, 1716, 1716, 1716, 1716, 494: 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 530: 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 570: 1716, 639: 1716, 642: 1716, 1716}, + {1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1187, 1716, 1716, 1716, 1716, 465: 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 476: 1716, 1716, 479: 1716, 1716, 1716, 1716, 1716, 485: 1716, 487: 1716, 1716, 1716, 1716, 492: 1716, 494: 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 530: 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 570: 1716, 639: 1716, 642: 1716, 1716}, {1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 652: 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715}, {1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 652: 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714}, // 280 {1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 652: 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713}, {1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 652: 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712}, {1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 652: 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711}, - {1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1190, 1710, 1710, 1710, 1710, 465: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 476: 1710, 1710, 479: 1710, 1710, 1710, 1710, 1710, 485: 1710, 487: 1710, 1710, 1710, 1710, 1710, 494: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 530: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 570: 1710, 639: 1710, 642: 1710, 1710}, + {1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1190, 1710, 1710, 1710, 1710, 465: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 476: 1710, 1710, 479: 1710, 1710, 1710, 1710, 1710, 485: 1710, 487: 1710, 1710, 1710, 1710, 492: 1710, 494: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 530: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 570: 1710, 639: 1710, 642: 1710, 1710}, {1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 652: 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709}, // 285 {1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 652: 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708}, @@ -6354,7 +6355,7 @@ var ( {1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 652: 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703}, {1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 652: 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702}, {1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 652: 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701}, - {1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 3957, 1700, 1700, 1700, 1700, 465: 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 476: 1700, 1700, 479: 1700, 1700, 1700, 1700, 1700, 485: 1700, 487: 1700, 1700, 1700, 1700, 1700, 494: 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 530: 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 570: 1700, 639: 1700, 642: 1700, 1700}, + {1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 3958, 1700, 1700, 1700, 1700, 465: 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 476: 1700, 1700, 479: 1700, 1700, 1700, 1700, 1700, 485: 1700, 487: 1700, 1700, 1700, 1700, 492: 1700, 494: 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 530: 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 570: 1700, 639: 1700, 642: 1700, 1700}, {1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 652: 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699}, // 295 {1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 652: 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698}, @@ -6384,28 +6385,28 @@ var ( {1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 652: 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678}, {1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 652: 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677}, {1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 652: 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676}, - {1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1174, 1675, 3956, 1675, 1675, 465: 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 476: 1675, 1675, 479: 1675, 1675, 1675, 1675, 1675, 485: 1675, 487: 1675, 1675, 1675, 1675, 1675, 494: 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 530: 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 570: 1675, 639: 1675, 642: 1675, 1675}, - {1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1173, 1674, 3955, 1674, 1674, 465: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 476: 1674, 1674, 479: 1674, 1674, 1674, 1674, 1674, 485: 1674, 487: 1674, 1674, 1674, 1674, 1674, 494: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 530: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 570: 1674, 639: 1674, 642: 1674, 1674}, + {1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1174, 1675, 3957, 1675, 1675, 465: 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 476: 1675, 1675, 479: 1675, 1675, 1675, 1675, 1675, 485: 1675, 487: 1675, 1675, 1675, 1675, 492: 1675, 494: 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 530: 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 570: 1675, 639: 1675, 642: 1675, 1675}, + {1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1173, 1674, 3956, 1674, 1674, 465: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 476: 1674, 1674, 479: 1674, 1674, 1674, 1674, 1674, 485: 1674, 487: 1674, 1674, 1674, 1674, 492: 1674, 494: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 530: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 570: 1674, 639: 1674, 642: 1674, 1674}, // 320 {1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 652: 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673}, {1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 652: 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672}, - {1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1172, 1671, 1671, 1671, 1671, 465: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 476: 1671, 1671, 479: 1671, 1671, 1671, 1671, 1671, 485: 1671, 487: 1671, 1671, 1671, 1671, 1671, 494: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 530: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 570: 1671, 639: 1671, 642: 1671, 1671}, + {1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1172, 1671, 1671, 1671, 1671, 465: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 476: 1671, 1671, 479: 1671, 1671, 1671, 1671, 1671, 485: 1671, 487: 1671, 1671, 1671, 1671, 492: 1671, 494: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 530: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 570: 1671, 639: 1671, 642: 1671, 1671}, {1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 652: 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670}, {1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 652: 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669}, // 325 {1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 652: 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668}, {1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 652: 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667}, - {1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1169, 1666, 1666, 1666, 1666, 465: 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 476: 1666, 1666, 479: 1666, 1666, 1666, 1666, 1666, 485: 1666, 487: 1666, 1666, 1666, 1666, 1666, 494: 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 530: 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 570: 1666, 639: 1666, 642: 1666, 1666}, + {1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1169, 1666, 1666, 1666, 1666, 465: 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 476: 1666, 1666, 479: 1666, 1666, 1666, 1666, 1666, 485: 1666, 487: 1666, 1666, 1666, 1666, 492: 1666, 494: 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 530: 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 570: 1666, 639: 1666, 642: 1666, 1666}, {1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 652: 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665}, - {1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1170, 1664, 1664, 1664, 1664, 465: 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 476: 1664, 1664, 479: 1664, 1664, 1664, 1664, 1664, 485: 1664, 487: 1664, 1664, 1664, 1664, 1664, 494: 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 530: 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 570: 1664, 639: 1664, 642: 1664, 1664}, + {1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1170, 1664, 1664, 1664, 1664, 465: 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 476: 1664, 1664, 479: 1664, 1664, 1664, 1664, 1664, 485: 1664, 487: 1664, 1664, 1664, 1664, 492: 1664, 494: 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 530: 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 570: 1664, 639: 1664, 642: 1664, 1664}, // 330 - {1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 3945, 1663, 1663, 1663, 1663, 465: 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 476: 1663, 1663, 479: 1663, 1663, 1663, 1663, 1663, 485: 1663, 487: 1663, 1663, 1663, 1663, 1663, 494: 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 530: 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 570: 1663, 639: 1663, 642: 1663, 1663}, + {1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 3946, 1663, 1663, 1663, 1663, 465: 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 476: 1663, 1663, 479: 1663, 1663, 1663, 1663, 1663, 485: 1663, 487: 1663, 1663, 1663, 1663, 492: 1663, 494: 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 530: 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 570: 1663, 639: 1663, 642: 1663, 1663}, {1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 652: 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662}, {1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 652: 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661}, - {1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1171, 1660, 1660, 1660, 1660, 465: 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 476: 1660, 1660, 479: 1660, 1660, 1660, 1660, 1660, 485: 1660, 487: 1660, 1660, 1660, 1660, 1660, 494: 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 530: 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 570: 1660, 639: 1660, 642: 1660, 1660}, + {1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1171, 1660, 1660, 1660, 1660, 465: 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 476: 1660, 1660, 479: 1660, 1660, 1660, 1660, 1660, 485: 1660, 487: 1660, 1660, 1660, 1660, 492: 1660, 494: 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 530: 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 570: 1660, 639: 1660, 642: 1660, 1660}, {1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 652: 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659}, // 335 - {1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1194, 1658, 1658, 1658, 1658, 465: 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 476: 1658, 1658, 479: 1658, 1658, 1658, 1658, 1658, 485: 1658, 487: 1658, 1658, 1658, 1658, 1658, 494: 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 530: 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 570: 1658, 639: 1658, 642: 1658, 1658}, + {1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1194, 1658, 1658, 1658, 1658, 465: 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 476: 1658, 1658, 479: 1658, 1658, 1658, 1658, 1658, 485: 1658, 487: 1658, 1658, 1658, 1658, 492: 1658, 494: 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 530: 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 570: 1658, 639: 1658, 642: 1658, 1658}, {1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 652: 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657}, {1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 652: 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656}, {1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 652: 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655}, @@ -6420,7 +6421,7 @@ var ( {1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 652: 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648}, {1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 652: 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647}, {1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 652: 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646}, - {1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1181, 1645, 1645, 1645, 1645, 465: 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 476: 1645, 1645, 479: 1645, 1645, 1645, 1645, 1645, 485: 1645, 487: 1645, 1645, 1645, 1645, 1645, 494: 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 530: 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 570: 1645, 639: 1645, 642: 1645, 1645}, + {1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1181, 1645, 1645, 1645, 1645, 465: 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 476: 1645, 1645, 479: 1645, 1645, 1645, 1645, 1645, 485: 1645, 487: 1645, 1645, 1645, 1645, 492: 1645, 494: 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 530: 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 570: 1645, 639: 1645, 642: 1645, 1645}, {1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 652: 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644}, // 350 {1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 652: 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643}, @@ -6445,7 +6446,7 @@ var ( {1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 652: 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627}, {1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 652: 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626}, {1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 652: 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625}, - {1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1178, 1624, 1624, 1624, 1624, 465: 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 476: 1624, 1624, 479: 1624, 1624, 1624, 1624, 1624, 485: 1624, 487: 1624, 1624, 1624, 1624, 1624, 494: 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 530: 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 570: 1624, 639: 1624, 642: 1624, 1624}, + {1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1178, 1624, 1624, 1624, 1624, 465: 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 476: 1624, 1624, 479: 1624, 1624, 1624, 1624, 1624, 485: 1624, 487: 1624, 1624, 1624, 1624, 492: 1624, 494: 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 530: 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 570: 1624, 639: 1624, 642: 1624, 1624}, // 370 {1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 652: 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623}, {1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 652: 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622}, @@ -6466,21 +6467,21 @@ var ( {1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 652: 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609}, // 385 {1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 652: 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608}, - {1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1176, 1607, 1607, 1607, 1607, 465: 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 476: 1607, 1607, 479: 1607, 1607, 1607, 1607, 1607, 485: 1607, 487: 1607, 1607, 1607, 1607, 1607, 494: 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 530: 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 570: 1607, 639: 1607, 642: 1607, 1607}, - {1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1195, 1606, 1606, 1606, 1606, 465: 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 476: 1606, 1606, 479: 1606, 1606, 1606, 1606, 1606, 485: 1606, 487: 1606, 1606, 1606, 1606, 1606, 494: 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 530: 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 570: 1606, 639: 1606, 642: 1606, 1606}, - {1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1183, 1605, 1605, 1605, 1605, 465: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 476: 1605, 1605, 479: 1605, 1605, 1605, 1605, 1605, 485: 1605, 487: 1605, 1605, 1605, 1605, 1605, 494: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 530: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 570: 1605, 639: 1605, 642: 1605, 1605}, + {1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1176, 1607, 1607, 1607, 1607, 465: 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 476: 1607, 1607, 479: 1607, 1607, 1607, 1607, 1607, 485: 1607, 487: 1607, 1607, 1607, 1607, 492: 1607, 494: 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 530: 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 570: 1607, 639: 1607, 642: 1607, 1607}, + {1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1195, 1606, 1606, 1606, 1606, 465: 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 476: 1606, 1606, 479: 1606, 1606, 1606, 1606, 1606, 485: 1606, 487: 1606, 1606, 1606, 1606, 492: 1606, 494: 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 530: 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 570: 1606, 639: 1606, 642: 1606, 1606}, + {1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1183, 1605, 1605, 1605, 1605, 465: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 476: 1605, 1605, 479: 1605, 1605, 1605, 1605, 1605, 485: 1605, 487: 1605, 1605, 1605, 1605, 492: 1605, 494: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 530: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 570: 1605, 639: 1605, 642: 1605, 1605}, {1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 652: 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604}, // 390 {1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 652: 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603}, {1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 652: 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602}, - {1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1185, 1601, 1601, 1601, 1601, 465: 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 476: 1601, 1601, 479: 1601, 1601, 1601, 1601, 1601, 485: 1601, 487: 1601, 1601, 1601, 1601, 1601, 494: 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 530: 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 570: 1601, 639: 1601, 642: 1601, 1601}, - {1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1184, 1600, 1600, 1600, 1600, 465: 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 476: 1600, 1600, 479: 1600, 1600, 1600, 1600, 1600, 485: 1600, 487: 1600, 1600, 1600, 1600, 1600, 494: 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 530: 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 570: 1600, 639: 1600, 642: 1600, 1600}, + {1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1185, 1601, 1601, 1601, 1601, 465: 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 476: 1601, 1601, 479: 1601, 1601, 1601, 1601, 1601, 485: 1601, 487: 1601, 1601, 1601, 1601, 492: 1601, 494: 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 530: 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 570: 1601, 639: 1601, 642: 1601, 1601}, + {1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1184, 1600, 1600, 1600, 1600, 465: 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 476: 1600, 1600, 479: 1600, 1600, 1600, 1600, 1600, 485: 1600, 487: 1600, 1600, 1600, 1600, 492: 1600, 494: 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 530: 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 570: 1600, 639: 1600, 642: 1600, 1600}, {1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 652: 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599}, // 395 {1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 652: 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598}, {1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 652: 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597}, {1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 652: 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596}, - {1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1175, 1595, 1595, 1595, 1595, 465: 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 476: 1595, 1595, 479: 1595, 1595, 1595, 1595, 1595, 485: 1595, 487: 1595, 1595, 1595, 1595, 1595, 494: 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 530: 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 570: 1595, 639: 1595, 642: 1595, 1595}, + {1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1175, 1595, 1595, 1595, 1595, 465: 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 476: 1595, 1595, 479: 1595, 1595, 1595, 1595, 1595, 485: 1595, 487: 1595, 1595, 1595, 1595, 492: 1595, 494: 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 530: 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 570: 1595, 639: 1595, 642: 1595, 1595}, {1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 652: 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594}, // 400 {1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 652: 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593}, @@ -6611,8 +6612,8 @@ var ( // 505 {1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 652: 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488}, {1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 652: 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487}, - {1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 3942, 1484, 1484, 1484, 1484, 465: 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 476: 1484, 1484, 479: 1484, 1484, 1484, 1484, 1484, 485: 1484, 487: 1484, 1484, 1484, 1484, 1484, 494: 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 530: 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 570: 1484, 639: 1484, 642: 1484, 1484}, - {1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 3931, 1483, 1483, 1483, 1483, 465: 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 476: 1483, 1483, 479: 1483, 1483, 1483, 1483, 1483, 485: 1483, 487: 1483, 1483, 1483, 1483, 1483, 494: 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 530: 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 570: 1483, 639: 1483, 642: 1483, 1483}, + {1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 3943, 1484, 1484, 1484, 1484, 465: 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 476: 1484, 1484, 479: 1484, 1484, 1484, 1484, 1484, 485: 1484, 487: 1484, 1484, 1484, 1484, 492: 1484, 494: 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 530: 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 570: 1484, 639: 1484, 642: 1484, 1484}, + {1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 3932, 1483, 1483, 1483, 1483, 465: 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 476: 1483, 1483, 479: 1483, 1483, 1483, 1483, 1483, 485: 1483, 487: 1483, 1483, 1483, 1483, 492: 1483, 494: 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 530: 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 570: 1483, 639: 1483, 642: 1483, 1483}, {1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 652: 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482}, // 510 {1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 652: 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481}, @@ -6726,7 +6727,7 @@ var ( {1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 652: 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391}, {1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 652: 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390}, {1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 652: 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389}, - {1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 3922, 1388, 1388, 1388, 1388, 465: 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 476: 1388, 1388, 479: 1388, 1388, 1388, 1388, 1388, 485: 1388, 487: 1388, 1388, 1388, 1388, 1388, 494: 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 530: 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 570: 1388, 639: 1388, 642: 1388, 1388}, + {1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 3923, 1388, 1388, 1388, 1388, 465: 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 476: 1388, 1388, 479: 1388, 1388, 1388, 1388, 1388, 485: 1388, 487: 1388, 1388, 1388, 1388, 492: 1388, 494: 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 530: 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 570: 1388, 639: 1388, 642: 1388, 1388}, {1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 652: 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387}, // 605 {1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 652: 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386}, @@ -6760,8 +6761,8 @@ var ( {1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 652: 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362}, // 630 {1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 652: 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361}, - {1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 3915, 1360, 1360, 1360, 1360, 465: 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 476: 1360, 1360, 479: 1360, 1360, 1360, 1360, 1360, 485: 1360, 487: 1360, 1360, 1360, 1360, 1360, 494: 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 530: 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 570: 1360, 639: 1360, 642: 1360, 1360}, - {1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 3908, 1359, 1359, 1359, 1359, 465: 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 476: 1359, 1359, 479: 1359, 1359, 1359, 1359, 1359, 485: 1359, 487: 1359, 1359, 1359, 1359, 1359, 494: 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 530: 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 570: 1359, 639: 1359, 642: 1359, 1359}, + {1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 3916, 1360, 1360, 1360, 1360, 465: 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 476: 1360, 1360, 479: 1360, 1360, 1360, 1360, 1360, 485: 1360, 487: 1360, 1360, 1360, 1360, 492: 1360, 494: 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 530: 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 570: 1360, 639: 1360, 642: 1360, 1360}, + {1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 3909, 1359, 1359, 1359, 1359, 465: 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 476: 1359, 1359, 479: 1359, 1359, 1359, 1359, 1359, 485: 1359, 487: 1359, 1359, 1359, 1359, 492: 1359, 494: 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 530: 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 570: 1359, 639: 1359, 642: 1359, 1359}, {1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 652: 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358}, {1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 652: 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357}, // 635 @@ -6784,8 +6785,8 @@ var ( {1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 652: 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342}, // 650 {1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 652: 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341}, - {1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 3888, 1340, 1340, 1340, 1340, 465: 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 476: 1340, 1340, 479: 1340, 1340, 1340, 1340, 1340, 485: 1340, 487: 1340, 1340, 1340, 1340, 1340, 494: 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 530: 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 570: 1340, 639: 1340, 642: 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, 3880, 1339, 1339, 1339, 1339, 465: 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 476: 1339, 1339, 479: 1339, 1339, 1339, 1339, 1339, 485: 1339, 487: 1339, 1339, 1339, 1339, 1339, 494: 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 530: 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 570: 1339, 639: 1339, 642: 1339, 1339}, + {1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 3889, 1340, 1340, 1340, 1340, 465: 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 476: 1340, 1340, 479: 1340, 1340, 1340, 1340, 1340, 485: 1340, 487: 1340, 1340, 1340, 1340, 492: 1340, 494: 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 530: 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 570: 1340, 639: 1340, 642: 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, 3881, 1339, 1339, 1339, 1339, 465: 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 476: 1339, 1339, 479: 1339, 1339, 1339, 1339, 1339, 485: 1339, 487: 1339, 1339, 1339, 1339, 492: 1339, 494: 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 530: 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 570: 1339, 639: 1339, 642: 1339, 1339}, {1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 652: 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338}, {1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 652: 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337}, // 655 @@ -6805,54 +6806,54 @@ var ( {1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 652: 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325}, {1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 652: 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324}, {1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 652: 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323}, - {1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 460: 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 485: 1285, 487: 1285, 1285, 1285, 1285, 1285, 494: 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 530: 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 563: 1285, 570: 1285, 1285, 573: 1285, 629: 1285, 1285, 1285, 633: 1285}, + {1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 460: 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 485: 1285, 487: 1285, 1285, 1285, 1285, 492: 1285, 494: 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 530: 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 563: 1285, 570: 1285, 1285, 573: 1285, 629: 1285, 1285, 1285, 633: 1285}, // 670 - {1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 460: 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 485: 1284, 487: 1284, 1284, 1284, 1284, 1284, 494: 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 530: 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 563: 1284, 570: 1284, 1284, 573: 1284, 629: 1284, 1284, 1284, 633: 1284}, - {1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 460: 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 485: 1283, 487: 1283, 1283, 1283, 1283, 1283, 494: 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 530: 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 563: 1283, 570: 1283, 1283, 573: 1283, 629: 1283, 1283, 1283, 633: 1283}, - {1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 460: 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 485: 1282, 487: 1282, 1282, 1282, 1282, 1282, 494: 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 530: 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 563: 1282, 570: 1282, 1282, 573: 1282, 629: 1282, 1282, 1282, 633: 1282}, - {1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 460: 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 485: 1281, 487: 1281, 1281, 1281, 1281, 1281, 494: 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 530: 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 563: 1281, 570: 1281, 1281, 573: 1281, 629: 1281, 1281, 1281, 633: 1281}, - {1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 460: 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 485: 1280, 487: 1280, 1280, 1280, 1280, 1280, 494: 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 530: 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 563: 1280, 570: 1280, 1280, 573: 1280, 629: 1280, 1280, 1280, 633: 1280}, + {1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 460: 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 485: 1284, 487: 1284, 1284, 1284, 1284, 492: 1284, 494: 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 530: 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 563: 1284, 570: 1284, 1284, 573: 1284, 629: 1284, 1284, 1284, 633: 1284}, + {1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 460: 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 485: 1283, 487: 1283, 1283, 1283, 1283, 492: 1283, 494: 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 530: 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 563: 1283, 570: 1283, 1283, 573: 1283, 629: 1283, 1283, 1283, 633: 1283}, + {1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 460: 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 485: 1282, 487: 1282, 1282, 1282, 1282, 492: 1282, 494: 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 530: 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 563: 1282, 570: 1282, 1282, 573: 1282, 629: 1282, 1282, 1282, 633: 1282}, + {1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 460: 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 485: 1281, 487: 1281, 1281, 1281, 1281, 492: 1281, 494: 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 530: 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 563: 1281, 570: 1281, 1281, 573: 1281, 629: 1281, 1281, 1281, 633: 1281}, + {1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 460: 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 485: 1280, 487: 1280, 1280, 1280, 1280, 492: 1280, 494: 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 530: 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 563: 1280, 570: 1280, 1280, 573: 1280, 629: 1280, 1280, 1280, 633: 1280}, // 675 - {1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 460: 1279, 3879, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 485: 1279, 487: 1279, 1279, 1279, 1279, 1279, 494: 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 530: 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 563: 1279, 570: 1279, 1279, 573: 1279, 629: 1279, 1279, 1279, 633: 1279}, - {461: 3876, 562: 3877, 566: 3878}, - {1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 460: 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 485: 1277, 487: 1277, 1277, 1277, 1277, 1277, 494: 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 530: 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 563: 1277, 570: 1277, 1277, 573: 1277, 629: 1277, 1277, 1277, 633: 1277}, - {1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 460: 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 485: 1276, 487: 1276, 1276, 1276, 1276, 1276, 494: 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 530: 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 563: 1276, 570: 1276, 1276, 573: 1276, 629: 1276, 1276, 1276, 633: 1276}, - {1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 460: 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 485: 1273, 487: 1273, 1273, 1273, 1273, 1273, 494: 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 530: 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 563: 1273, 570: 1273, 1273, 573: 1273, 629: 1273, 1273, 1273, 633: 1273}, + {1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 460: 1279, 3880, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 485: 1279, 487: 1279, 1279, 1279, 1279, 492: 1279, 494: 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 530: 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 563: 1279, 570: 1279, 1279, 573: 1279, 629: 1279, 1279, 1279, 633: 1279}, + {461: 3877, 562: 3878, 566: 3879}, + {1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 460: 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 485: 1277, 487: 1277, 1277, 1277, 1277, 492: 1277, 494: 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 530: 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 1277, 563: 1277, 570: 1277, 1277, 573: 1277, 629: 1277, 1277, 1277, 633: 1277}, + {1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 460: 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 485: 1276, 487: 1276, 1276, 1276, 1276, 492: 1276, 494: 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 530: 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 1276, 563: 1276, 570: 1276, 1276, 573: 1276, 629: 1276, 1276, 1276, 633: 1276}, + {1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 460: 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 485: 1273, 487: 1273, 1273, 1273, 1273, 492: 1273, 494: 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 530: 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 1273, 563: 1273, 570: 1273, 1273, 573: 1273, 629: 1273, 1273, 1273, 633: 1273}, // 680 - {1268, 1268, 7: 3298, 57: 1268, 124: 1268, 458: 1268, 460: 1268, 466: 1268, 468: 1268, 476: 1268, 1268, 479: 1268, 1268, 1268, 483: 1268, 488: 1268}, + {1268, 1268, 7: 3299, 57: 1268, 124: 1268, 458: 1268, 460: 1268, 466: 1268, 468: 1268, 476: 1268, 1268, 479: 1268, 1268, 1268, 483: 1268, 488: 1268}, {1267, 1267, 7: 1267, 57: 1267, 124: 1267, 458: 1267, 460: 1267, 466: 1267, 468: 1267, 476: 1267, 1267, 479: 1267, 1267, 1267, 483: 1267, 488: 1267, 490: 1267, 501: 1267, 1267, 510: 1267, 513: 1267, 1267}, - {1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 460: 1242, 1242, 1242, 1242, 465: 1242, 1242, 3238, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 476: 1242, 1242, 479: 1242, 1242, 1242, 1242, 1242, 485: 1242, 487: 1242, 1242, 1242, 1242, 1242, 494: 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 530: 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 570: 3239}, - {1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 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, 1241, 1241, 1241, 1241, 1241, 1241, 476: 1241, 1241, 479: 1241, 1241, 1241, 1241, 1241, 485: 1241, 487: 1241, 1241, 1241, 1241, 1241, 494: 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 530: 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 570: 1241, 639: 3871, 642: 1241, 1241}, - {1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 460: 1238, 1238, 1238, 1238, 465: 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 476: 1238, 1238, 479: 1238, 1238, 1238, 1238, 1238, 485: 1238, 487: 1238, 1238, 1238, 1238, 1238, 494: 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 530: 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 570: 1238, 642: 3867, 3868}, + {1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 460: 1242, 1242, 1242, 1242, 465: 1242, 1242, 3239, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 476: 1242, 1242, 479: 1242, 1242, 1242, 1242, 1242, 485: 1242, 487: 1242, 1242, 1242, 1242, 492: 1242, 494: 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 530: 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 570: 3240}, + {1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 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, 1241, 1241, 1241, 1241, 1241, 1241, 476: 1241, 1241, 479: 1241, 1241, 1241, 1241, 1241, 485: 1241, 487: 1241, 1241, 1241, 1241, 492: 1241, 494: 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 530: 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 570: 1241, 639: 3872, 642: 1241, 1241}, + {1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 460: 1238, 1238, 1238, 1238, 465: 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 476: 1238, 1238, 479: 1238, 1238, 1238, 1238, 1238, 485: 1238, 487: 1238, 1238, 1238, 1238, 492: 1238, 494: 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 530: 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 570: 1238, 642: 3868, 3869}, // 685 - {1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 460: 1237, 1237, 1237, 1237, 465: 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 476: 1237, 1237, 479: 1237, 1237, 1237, 1237, 1237, 485: 1237, 487: 1237, 1237, 1237, 1237, 1237, 494: 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 530: 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 570: 1237}, - {1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 460: 1236, 1236, 1236, 1236, 465: 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 476: 1236, 1236, 479: 1236, 1236, 1236, 1236, 1236, 485: 1236, 487: 1236, 1236, 1236, 1236, 1236, 494: 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 530: 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 570: 1236}, - {1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 460: 1235, 1235, 1235, 1235, 465: 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 476: 1235, 1235, 479: 1235, 1235, 1235, 1235, 1235, 485: 1235, 487: 1235, 1235, 1235, 1235, 1235, 494: 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 530: 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 570: 1235}, - {1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 460: 1233, 1233, 1233, 1233, 465: 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 476: 1233, 1233, 479: 1233, 1233, 1233, 1233, 1233, 485: 1233, 487: 1233, 1233, 1233, 1233, 1233, 494: 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 530: 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 570: 1233}, - {1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 460: 1232, 1232, 1232, 1232, 465: 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 476: 1232, 1232, 479: 1232, 1232, 1232, 1232, 1232, 485: 1232, 487: 1232, 1232, 1232, 1232, 1232, 494: 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 530: 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 570: 1232}, + {1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 460: 1237, 1237, 1237, 1237, 465: 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 476: 1237, 1237, 479: 1237, 1237, 1237, 1237, 1237, 485: 1237, 487: 1237, 1237, 1237, 1237, 492: 1237, 494: 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 530: 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 570: 1237}, + {1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 460: 1236, 1236, 1236, 1236, 465: 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 476: 1236, 1236, 479: 1236, 1236, 1236, 1236, 1236, 485: 1236, 487: 1236, 1236, 1236, 1236, 492: 1236, 494: 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 530: 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 570: 1236}, + {1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 460: 1235, 1235, 1235, 1235, 465: 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 476: 1235, 1235, 479: 1235, 1235, 1235, 1235, 1235, 485: 1235, 487: 1235, 1235, 1235, 1235, 492: 1235, 494: 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 530: 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 570: 1235}, + {1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 460: 1233, 1233, 1233, 1233, 465: 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 476: 1233, 1233, 479: 1233, 1233, 1233, 1233, 1233, 485: 1233, 487: 1233, 1233, 1233, 1233, 492: 1233, 494: 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 530: 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 570: 1233}, + {1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 460: 1232, 1232, 1232, 1232, 465: 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 476: 1232, 1232, 479: 1232, 1232, 1232, 1232, 1232, 485: 1232, 487: 1232, 1232, 1232, 1232, 492: 1232, 494: 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 530: 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 570: 1232}, // 690 - {1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 460: 1231, 1231, 1231, 1231, 465: 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 476: 1231, 1231, 479: 1231, 1231, 1231, 1231, 1231, 485: 1231, 487: 1231, 1231, 1231, 1231, 1231, 494: 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 530: 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 570: 1231}, - {1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 460: 1230, 1230, 1230, 1230, 465: 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 476: 1230, 1230, 479: 1230, 1230, 1230, 1230, 1230, 485: 1230, 487: 1230, 1230, 1230, 1230, 1230, 494: 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 530: 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 570: 1230}, - {1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 460: 1229, 1229, 1229, 1229, 465: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 476: 1229, 1229, 479: 1229, 1229, 1229, 1229, 1229, 485: 1229, 487: 1229, 1229, 1229, 1229, 1229, 494: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 530: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 570: 1229}, - {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, 3866, 3140, 3223, 3139, 3136}, - {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, 3865, 3140, 3223, 3139, 3136}, + {1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 460: 1231, 1231, 1231, 1231, 465: 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 476: 1231, 1231, 479: 1231, 1231, 1231, 1231, 1231, 485: 1231, 487: 1231, 1231, 1231, 1231, 492: 1231, 494: 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 530: 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 570: 1231}, + {1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 460: 1230, 1230, 1230, 1230, 465: 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 476: 1230, 1230, 479: 1230, 1230, 1230, 1230, 1230, 485: 1230, 487: 1230, 1230, 1230, 1230, 492: 1230, 494: 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 530: 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 570: 1230}, + {1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 460: 1229, 1229, 1229, 1229, 465: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 476: 1229, 1229, 479: 1229, 1229, 1229, 1229, 1229, 485: 1229, 487: 1229, 1229, 1229, 1229, 492: 1229, 494: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 530: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 570: 1229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3867, 3141, 3224, 3140, 3137}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3866, 3141, 3224, 3140, 3137}, // 695 - {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, 3864, 3140, 3223, 3139, 3136}, - {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, 3863, 3140, 3223, 3139, 3136}, - {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, 3862, 3140, 3223, 3139, 3136}, - {1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 460: 1222, 1222, 1222, 1222, 465: 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 476: 1222, 1222, 479: 1222, 1222, 1222, 1222, 1222, 485: 1222, 487: 1222, 1222, 1222, 1222, 1222, 494: 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 530: 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 570: 1222}, - {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, 2496, 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3763, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 2494, 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, 632: 2490, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3762, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3760, 738: 3765, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 3767, 3766, 3764, 761: 3761}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3865, 3141, 3224, 3140, 3137}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3864, 3141, 3224, 3140, 3137}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3863, 3141, 3224, 3140, 3137}, + {1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 460: 1222, 1222, 1222, 1222, 465: 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 476: 1222, 1222, 479: 1222, 1222, 1222, 1222, 1222, 485: 1222, 487: 1222, 1222, 1222, 1222, 492: 1222, 494: 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 530: 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 570: 1222}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 2497, 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3764, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 2495, 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 632: 2491, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3763, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3761, 738: 3766, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 3768, 3767, 3765, 761: 3762}, // 700 - {459: 3755}, - {459: 2497, 696: 3754}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 3751, 2664, 2665, 2663}, - {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}, + {459: 3756}, + {459: 2498, 696: 3755}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3752, 2665, 2666, 2664}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3751, 3141, 3224, 3140, 3137}, + {459: 3746}, // 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, 1183: 3733}, - {459: 3674}, - {459: 3671}, - {459: 3663}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 528: 1043, 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3733, 1183: 3734}, + {459: 3675}, + {459: 3672}, + {459: 3664}, {459: 1192}, // 710 {459: 1189}, @@ -6863,110 +6864,110 @@ var ( // 715 {459: 1179}, {459: 1177}, - {1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 465: 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 476: 1166, 1166, 479: 1166, 1166, 1166, 1166, 1166, 485: 1166, 487: 1166, 1166, 1166, 1166, 1166, 494: 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 530: 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 570: 1166}, - {1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 465: 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 476: 1165, 1165, 479: 1165, 1165, 1165, 1165, 1165, 485: 1165, 487: 1165, 1165, 1165, 1165, 1165, 494: 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 530: 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 570: 1165}, - {1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 465: 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 476: 1164, 1164, 479: 1164, 1164, 1164, 1164, 1164, 485: 1164, 487: 1164, 1164, 1164, 1164, 1164, 494: 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 530: 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 570: 1164}, + {1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 465: 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 476: 1166, 1166, 479: 1166, 1166, 1166, 1166, 1166, 485: 1166, 487: 1166, 1166, 1166, 1166, 492: 1166, 494: 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 530: 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 570: 1166}, + {1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 465: 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 476: 1165, 1165, 479: 1165, 1165, 1165, 1165, 1165, 485: 1165, 487: 1165, 1165, 1165, 1165, 492: 1165, 494: 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 530: 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 570: 1165}, + {1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 465: 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 476: 1164, 1164, 479: 1164, 1164, 1164, 1164, 1164, 485: 1164, 487: 1164, 1164, 1164, 1164, 492: 1164, 494: 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 530: 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 570: 1164}, // 720 - {1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 465: 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 476: 1163, 1163, 479: 1163, 1163, 1163, 1163, 1163, 485: 1163, 487: 1163, 1163, 1163, 1163, 1163, 494: 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 530: 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 570: 1163}, - {1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 465: 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 476: 1162, 1162, 479: 1162, 1162, 1162, 1162, 1162, 485: 1162, 487: 1162, 1162, 1162, 1162, 1162, 494: 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 530: 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 570: 1162}, - {1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 465: 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 476: 1161, 1161, 479: 1161, 1161, 1161, 1161, 1161, 485: 1161, 487: 1161, 1161, 1161, 1161, 1161, 494: 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 530: 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 570: 1161}, - {1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 465: 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 476: 1160, 1160, 479: 1160, 1160, 1160, 1160, 1160, 485: 1160, 487: 1160, 1160, 1160, 1160, 1160, 494: 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 530: 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 570: 1160}, - {1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 465: 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 476: 1159, 1159, 479: 1159, 1159, 1159, 1159, 1159, 485: 1159, 487: 1159, 1159, 1159, 1159, 1159, 494: 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 530: 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 570: 1159}, + {1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 465: 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 476: 1163, 1163, 479: 1163, 1163, 1163, 1163, 1163, 485: 1163, 487: 1163, 1163, 1163, 1163, 492: 1163, 494: 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 530: 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 570: 1163}, + {1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 465: 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 476: 1162, 1162, 479: 1162, 1162, 1162, 1162, 1162, 485: 1162, 487: 1162, 1162, 1162, 1162, 492: 1162, 494: 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 530: 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 570: 1162}, + {1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 465: 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 476: 1161, 1161, 479: 1161, 1161, 1161, 1161, 1161, 485: 1161, 487: 1161, 1161, 1161, 1161, 492: 1161, 494: 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 530: 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 570: 1161}, + {1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 465: 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 476: 1160, 1160, 479: 1160, 1160, 1160, 1160, 1160, 485: 1160, 487: 1160, 1160, 1160, 1160, 492: 1160, 494: 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 530: 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 570: 1160}, + {1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 465: 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 476: 1159, 1159, 479: 1159, 1159, 1159, 1159, 1159, 485: 1159, 487: 1159, 1159, 1159, 1159, 492: 1159, 494: 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 530: 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 570: 1159}, // 725 - {1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 465: 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 476: 1158, 1158, 479: 1158, 1158, 1158, 1158, 1158, 485: 1158, 487: 1158, 1158, 1158, 1158, 1158, 494: 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 530: 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 570: 1158}, - {1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 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, 1070: 3655}, + {1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 465: 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 476: 1158, 1158, 479: 1158, 1158, 1158, 1158, 1158, 485: 1158, 487: 1158, 1158, 1158, 1158, 492: 1158, 494: 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 530: 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 570: 1158}, + {1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 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, 492: 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: 3661}, + {459: 3658}, + {1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 3655, 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, 492: 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, 1071: 3656}, // 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, 1196: 3647}, - {459: 3639}, - {459: 3635}, - {459: 3630}, + {459: 3653}, + {1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 3649, 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, 492: 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: 3648}, + {459: 3640}, + {459: 3636}, + {459: 3631}, // 735 - {459: 3627}, - {459: 3622}, - {459: 3613}, - {459: 3606}, - {459: 3601}, + {459: 3628}, + {459: 3623}, + {459: 3614}, + {459: 3607}, + {459: 3602}, // 740 - {459: 3566}, - {459: 3552}, - {459: 3535}, - {1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 460: 1122, 1122, 1122, 1122, 465: 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 476: 1122, 1122, 479: 1122, 1122, 1122, 1122, 1122, 485: 1122, 487: 1122, 1122, 1122, 1122, 1122, 494: 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 530: 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 570: 1122}, - {459: 3528}, + {459: 3567}, + {459: 3553}, + {459: 3536}, + {1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 460: 1122, 1122, 1122, 1122, 465: 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 476: 1122, 1122, 479: 1122, 1122, 1122, 1122, 1122, 485: 1122, 487: 1122, 1122, 1122, 1122, 492: 1122, 494: 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 530: 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 1122, 570: 1122}, + {459: 3529}, // 745 {459: 1116}, {459: 1115}, {459: 1114}, {459: 1113}, - {1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 460: 1107, 1107, 1107, 1107, 465: 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 476: 1107, 1107, 479: 1107, 1107, 1107, 1107, 1107, 485: 1107, 487: 1107, 1107, 1107, 1107, 1107, 494: 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 530: 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 570: 1107}, + {1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 460: 1107, 1107, 1107, 1107, 465: 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 476: 1107, 1107, 479: 1107, 1107, 1107, 1107, 1107, 485: 1107, 487: 1107, 1107, 1107, 1107, 492: 1107, 494: 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 530: 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 570: 1107}, // 750 - {459: 3525}, - {459: 3522}, - {459: 3514}, - {459: 3506}, - {459: 3498}, + {459: 3526}, + {459: 3523}, + {459: 3515}, + {459: 3507}, + {459: 3499}, // 755 - {459: 3484}, - {459: 3472}, - {459: 3467}, - {459: 3462}, - {459: 3457}, + {459: 3485}, + {459: 3473}, + {459: 3468}, + {459: 3463}, + {459: 3458}, // 760 - {459: 3452}, - {459: 3447}, - {459: 3442}, - {459: 3429}, - {459: 3426}, + {459: 3453}, + {459: 3448}, + {459: 3443}, + {459: 3430}, + {459: 3427}, // 765 - {459: 3423}, - {459: 3420}, - {459: 3417}, - {459: 3414}, - {459: 3410}, + {459: 3424}, + {459: 3421}, + {459: 3418}, + {459: 3415}, + {459: 3411}, // 770 - {459: 3404}, - {459: 3391}, - {459: 3386}, - {459: 3381}, - {459: 3226}, + {459: 3405}, + {459: 3392}, + {459: 3387}, + {459: 3382}, + {459: 3227}, // 775 - {718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 460: 718, 718, 718, 718, 465: 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 476: 718, 718, 479: 718, 718, 718, 718, 718, 485: 718, 487: 718, 718, 718, 718, 718, 494: 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 530: 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 570: 718}, - {717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 460: 717, 717, 717, 717, 465: 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 476: 717, 717, 479: 717, 717, 717, 717, 717, 485: 717, 487: 717, 717, 717, 717, 717, 494: 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 530: 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 570: 717}, - {716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 460: 716, 716, 716, 716, 465: 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 476: 716, 716, 479: 716, 716, 716, 716, 716, 485: 716, 487: 716, 716, 716, 716, 716, 494: 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 530: 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 570: 716}, - {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, 3227}, - {7: 3235, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, + {718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 460: 718, 718, 718, 718, 465: 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 476: 718, 718, 479: 718, 718, 718, 718, 718, 485: 718, 487: 718, 718, 718, 718, 492: 718, 494: 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 530: 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 718, 570: 718}, + {717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 460: 717, 717, 717, 717, 465: 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 476: 717, 717, 479: 717, 717, 717, 717, 717, 485: 717, 487: 717, 717, 717, 717, 492: 717, 494: 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 530: 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 717, 570: 717}, + {716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 460: 716, 716, 716, 716, 465: 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 476: 716, 716, 479: 716, 716, 716, 716, 716, 485: 716, 487: 716, 716, 716, 716, 492: 716, 494: 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 530: 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 716, 570: 716}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3228}, + {7: 3236, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, // 780 - {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, 3380}, - {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, 3379}, - {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, 3378}, - {2: 1874, 1874, 1874, 1874, 1874, 8: 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 58: 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 459: 1874, 461: 1874, 1874, 1874, 1874, 469: 1874, 1874, 1874, 1874, 1874, 478: 1874, 484: 1874, 486: 1874, 492: 1874, 1874, 529: 1874, 552: 1874, 554: 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 564: 1874, 1874, 1874, 1874, 1874, 1874, 572: 1874, 574: 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 636: 1874}, - {2: 1873, 1873, 1873, 1873, 1873, 8: 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 58: 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 459: 1873, 461: 1873, 1873, 1873, 1873, 469: 1873, 1873, 1873, 1873, 1873, 478: 1873, 484: 1873, 486: 1873, 492: 1873, 1873, 529: 1873, 552: 1873, 554: 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 564: 1873, 1873, 1873, 1873, 1873, 1873, 572: 1873, 574: 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 636: 1873}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3381}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3380}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3379}, + {2: 1874, 1874, 1874, 1874, 1874, 8: 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 58: 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 459: 1874, 461: 1874, 1874, 1874, 1874, 469: 1874, 1874, 1874, 1874, 1874, 478: 1874, 484: 1874, 486: 1874, 491: 1874, 493: 1874, 529: 1874, 552: 1874, 554: 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 564: 1874, 1874, 1874, 1874, 1874, 1874, 572: 1874, 574: 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 1874, 636: 1874}, + {2: 1873, 1873, 1873, 1873, 1873, 8: 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 58: 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 459: 1873, 461: 1873, 1873, 1873, 1873, 469: 1873, 1873, 1873, 1873, 1873, 478: 1873, 484: 1873, 486: 1873, 491: 1873, 493: 1873, 529: 1873, 552: 1873, 554: 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 564: 1873, 1873, 1873, 1873, 1873, 1873, 572: 1873, 574: 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 1873, 636: 1873}, // 785 - {2: 1872, 1872, 1872, 1872, 1872, 8: 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 58: 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 459: 1872, 461: 1872, 1872, 1872, 1872, 469: 1872, 1872, 1872, 1872, 1872, 478: 1872, 484: 1872, 486: 1872, 492: 1872, 1872, 529: 1872, 552: 1872, 554: 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 564: 1872, 1872, 1872, 1872, 1872, 1872, 572: 1872, 574: 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 636: 1872}, - {2: 1871, 1871, 1871, 1871, 1871, 8: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 58: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 459: 1871, 461: 1871, 1871, 1871, 1871, 469: 1871, 1871, 1871, 1871, 1871, 478: 1871, 484: 1871, 486: 1871, 492: 1871, 1871, 529: 1871, 552: 1871, 554: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 564: 1871, 1871, 1871, 1871, 1871, 1871, 572: 1871, 574: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 636: 1871}, - {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, 3236, 3140, 3223, 3139, 3136}, - {57: 3240, 467: 3238, 570: 3239}, - {715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 460: 715, 715, 715, 715, 465: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 476: 715, 715, 479: 715, 715, 715, 715, 715, 485: 715, 487: 715, 715, 715, 715, 715, 494: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 530: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 570: 715}, + {2: 1872, 1872, 1872, 1872, 1872, 8: 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 58: 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 459: 1872, 461: 1872, 1872, 1872, 1872, 469: 1872, 1872, 1872, 1872, 1872, 478: 1872, 484: 1872, 486: 1872, 491: 1872, 493: 1872, 529: 1872, 552: 1872, 554: 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 564: 1872, 1872, 1872, 1872, 1872, 1872, 572: 1872, 574: 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 1872, 636: 1872}, + {2: 1871, 1871, 1871, 1871, 1871, 8: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 58: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 459: 1871, 461: 1871, 1871, 1871, 1871, 469: 1871, 1871, 1871, 1871, 1871, 478: 1871, 484: 1871, 486: 1871, 491: 1871, 493: 1871, 529: 1871, 552: 1871, 554: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 564: 1871, 1871, 1871, 1871, 1871, 1871, 572: 1871, 574: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 636: 1871}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3237, 3141, 3224, 3140, 3137}, + {57: 3241, 467: 3239, 570: 3240}, + {715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 460: 715, 715, 715, 715, 465: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 476: 715, 715, 479: 715, 715, 715, 715, 715, 485: 715, 487: 715, 715, 715, 715, 492: 715, 494: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 530: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 570: 715}, // 790 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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, 864: 3245}, - {8: 3243, 336: 3244}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 529: 3376, 648: 3378, 2665, 2666, 2664, 725: 3375, 853: 3374}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3373, 3141, 3224, 3140, 3137}, + {144: 900, 474: 900, 487: 3243, 727: 900, 1237: 3242}, + {144: 3247, 474: 3248, 727: 903, 867: 3246}, + {8: 3244, 336: 3245}, // 795 {144: 899, 474: 899, 727: 899}, {144: 898, 474: 898, 727: 898}, - {727: 3250, 734: 3251}, - {257: 3249}, - {257: 3248}, + {727: 3251, 734: 3252}, + {258: 3250}, + {258: 3249}, // 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, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 3288, 648: 3287, 2665, 2666, 2664, 907: 3290, 1139: 3291, 1320: 3289}, + {909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 909, 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, 492: 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 {1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 652: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773}, @@ -7008,59 +7009,59 @@ var ( {1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 652: 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359}, {1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 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, 907: 3293, 1236: 3292, 1321: 3291}, + {951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 951, 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, 492: 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: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 948, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 475: 948, 490: 948, 510: 948, 513: 948, 948, 648: 3287, 2665, 2666, 2664, 907: 3294, 1236: 3293, 1321: 3292}, // 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}, - {920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 460: 920, 920, 920, 920, 465: 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 476: 920, 920, 479: 920, 920, 920, 920, 920, 485: 920, 487: 920, 920, 920, 920, 920, 494: 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 530: 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 570: 920}, - {57: 3371}, - {57: 946, 475: 3295, 490: 946, 510: 946, 513: 946, 946, 1239: 3294}, + {922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 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, 492: 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, 492: 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}, + {920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 460: 920, 920, 920, 920, 465: 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 476: 920, 920, 479: 920, 920, 920, 920, 920, 485: 920, 487: 920, 920, 920, 920, 492: 920, 494: 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 530: 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 570: 920}, + {57: 3372}, + {57: 946, 475: 3296, 490: 946, 510: 946, 513: 946, 946, 1239: 3295}, // 845 {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, 881: 3297}, - {7: 3298, 57: 945, 490: 945, 510: 945, 513: 945, 945}, + {57: 944, 490: 3302, 510: 944, 513: 944, 944, 1242: 3301}, + {646: 3297}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 2657, 852: 3130, 885: 3298}, + {7: 3299, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 2657, 852: 3300}, {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, 881: 3303}, + {57: 942, 510: 3307, 513: 3308, 3309, 1241: 3305, 1319: 3306}, + {646: 3303}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 2657, 852: 3130, 885: 3304}, // 855 - {7: 3298, 57: 943, 510: 943, 513: 943, 943}, + {7: 3299, 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, 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}, + {145: 3320, 159: 3316, 493: 3310, 541: 3321, 559: 3312, 3311, 564: 3318, 567: 3319, 806: 3317, 961: 3314, 1317: 3315, 3313}, + {145: 940, 159: 940, 493: 940, 541: 940, 559: 940, 940, 564: 940, 567: 940}, + {145: 939, 159: 939, 493: 939, 541: 939, 559: 939, 939, 564: 939, 567: 939}, // 860 - {145: 938, 157: 938, 493: 938, 541: 938, 559: 938, 938, 564: 938, 567: 938}, - {2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 130: 2149, 147: 2149, 458: 2149, 2149, 2149, 462: 2149, 2149, 2149, 2149, 2149, 2149, 474: 2149, 2149, 478: 2149, 484: 2149, 2149, 2149, 492: 2149, 553: 2149, 563: 2149, 571: 2149, 573: 2149, 629: 2149, 2149, 2149, 2149, 2149, 2149, 2149}, - {2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 130: 2148, 147: 2148, 184: 2148, 458: 2148, 2148, 2148, 462: 2148, 2148, 2148, 2148, 2148, 2148, 474: 2148, 2148, 478: 2148, 484: 2148, 2148, 2148, 492: 2148, 553: 2148, 563: 2148, 571: 2148, 573: 2148, 629: 2148, 2148, 2148, 2148, 2148, 2148, 2148}, - {2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 2147, 130: 2147, 147: 2147, 184: 2147, 458: 2147, 2147, 2147, 462: 2147, 2147, 2147, 2147, 2147, 2147, 474: 2147, 2147, 478: 2147, 484: 2147, 2147, 2147, 492: 2147, 553: 2147, 563: 2147, 571: 2147, 573: 2147, 629: 2147, 2147, 2147, 2147, 2147, 2147, 2147}, + {145: 938, 159: 938, 493: 938, 541: 938, 559: 938, 938, 564: 938, 567: 938}, + {2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 130: 2150, 147: 2150, 458: 2150, 2150, 2150, 462: 2150, 2150, 2150, 2150, 2150, 2150, 474: 2150, 2150, 478: 2150, 484: 2150, 2150, 2150, 491: 2150, 553: 2150, 563: 2150, 571: 2150, 573: 2150, 629: 2150, 2150, 2150, 2150, 2150, 2150, 2150}, + {2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 2149, 130: 2149, 147: 2149, 184: 2149, 458: 2149, 2149, 2149, 462: 2149, 2149, 2149, 2149, 2149, 2149, 474: 2149, 2149, 478: 2149, 484: 2149, 2149, 2149, 491: 2149, 553: 2149, 563: 2149, 571: 2149, 573: 2149, 629: 2149, 2149, 2149, 2149, 2149, 2149, 2149}, + {2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 2148, 130: 2148, 147: 2148, 184: 2148, 458: 2148, 2148, 2148, 462: 2148, 2148, 2148, 2148, 2148, 2148, 474: 2148, 2148, 478: 2148, 484: 2148, 2148, 2148, 491: 2148, 553: 2148, 563: 2148, 571: 2148, 573: 2148, 629: 2148, 2148, 2148, 2148, 2148, 2148, 2148}, {57: 941}, // 865 {57: 937}, {57: 936}, - {130: 3366}, - {130: 3364}, - {130: 3362}, + {130: 3367}, + {130: 3365}, + {130: 3363}, // 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, 960: 3326, 1138: 3325}, - {130: 3366, 147: 3367}, - {130: 3364, 147: 3365}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3370}, + {561: 3369}, + {145: 3320, 159: 3322, 493: 3310, 559: 3312, 3311, 564: 3324, 567: 3325, 806: 3323, 961: 3327, 1138: 3326}, + {130: 3367, 147: 3368}, + {130: 3365, 147: 3366}, // 875 - {130: 3362, 147: 3363}, - {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, 960: 3326, 1138: 3328}, + {130: 3363, 147: 3364}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3330}, + {492: 3328}, + {57: 929, 492: 929}, + {145: 3320, 159: 3322, 493: 3310, 559: 3312, 3311, 564: 3324, 567: 3325, 806: 3323, 961: 3327, 1138: 3329}, // 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, 792: 3330, 816: 3359}, + {105: 3351, 3347, 108: 3344, 3359, 111: 3346, 3343, 3345, 3349, 3350, 3355, 3354, 3353, 3357, 3358, 3352, 3356, 3348, 492: 3235, 494: 3233, 3234, 3232, 3230, 517: 3341, 3338, 3340, 3339, 3335, 3337, 3336, 3333, 3334, 3332, 3342, 720: 3231, 3229, 793: 3331, 816: 3360}, {1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 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}, @@ -7096,420 +7097,420 @@ var ( {1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 465: 1045, 1045, 468: 1045, 1045, 1045, 1045, 1045, 1045, 1045, 476: 1045, 1045, 479: 1045, 1045, 1045, 1045, 1045, 1045, 1045, 487: 1045, 1045, 1045, 1045, 1045, 1045, 494: 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 530: 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 1045, 553: 1045, 632: 1045}, // 910 {1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 465: 1044, 1044, 468: 1044, 1044, 1044, 1044, 1044, 1044, 1044, 476: 1044, 1044, 479: 1044, 1044, 1044, 1044, 1044, 1044, 1044, 487: 1044, 1044, 1044, 1044, 1044, 1044, 494: 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 530: 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 1044, 553: 1044, 632: 1044}, - {130: 3360, 147: 3361}, - {57: 932, 491: 932}, - {57: 925, 491: 925}, - {57: 933, 491: 933}, + {130: 3361, 147: 3362}, + {57: 932, 492: 932}, + {57: 925, 492: 925}, + {57: 933, 492: 933}, // 915 - {57: 926, 491: 926}, - {57: 934, 491: 934}, - {57: 927, 491: 927}, - {57: 935, 491: 935}, - {57: 928, 491: 928}, + {57: 926, 492: 926}, + {57: 934, 492: 934}, + {57: 927, 492: 927}, + {57: 935, 492: 935}, + {57: 928, 492: 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, 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}, + {57: 931, 492: 931}, + {105: 3351, 3347, 108: 3344, 3359, 111: 3346, 3343, 3345, 3349, 3350, 3355, 3354, 3353, 3357, 3358, 3352, 3356, 3348, 492: 3235, 494: 3233, 3234, 3232, 3230, 517: 3341, 3338, 3340, 3339, 3335, 3337, 3336, 3333, 3334, 3332, 3342, 720: 3231, 3229, 793: 3331, 816: 3371}, + {130: 3361}, + {950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 950, 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, 492: 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, 3239, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 476: 1224, 1224, 479: 1224, 1224, 1224, 1224, 1224, 485: 1224, 487: 1224, 1224, 1224, 1224, 492: 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}, // 925 - {1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 460: 1234, 1234, 1234, 1234, 465: 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 476: 1234, 1234, 479: 1234, 1234, 1234, 1234, 1234, 485: 1234, 487: 1234, 1234, 1234, 1234, 1234, 494: 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 530: 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 570: 1234}, + {1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 460: 1234, 1234, 1234, 1234, 465: 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 476: 1234, 1234, 479: 1234, 1234, 1234, 1234, 1234, 485: 1234, 487: 1234, 1234, 1234, 1234, 492: 1234, 494: 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 530: 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 570: 1234}, {722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 494: 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 530: 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 553: 722, 563: 722, 570: 722, 722, 573: 722, 629: 722, 722, 722, 722, 722, 722, 722}, {721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 494: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 530: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 553: 721, 563: 721, 570: 721, 721, 573: 721, 629: 721, 721, 721, 721, 721, 721, 721}, {248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 494: 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 553: 248, 248, 563: 248, 570: 248, 248, 573: 248, 629: 248, 248, 248, 248, 248, 248, 248, 637: 248, 640: 248, 645: 248, 248, 652: 248, 654: 248, 248}, {247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 494: 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 553: 247, 247, 563: 247, 570: 247, 247, 573: 247, 629: 247, 247, 247, 247, 247, 247, 247, 637: 247, 640: 247, 645: 247, 247, 652: 247, 654: 247, 247}, // 930 - {1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 460: 1888, 1888, 465: 1888, 1888, 468: 1888, 1888, 1888, 474: 1888, 476: 1888, 1888, 479: 1888, 1888, 1888, 483: 1888, 485: 1888, 487: 1888, 1888, 1888, 1888, 1888, 494: 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 508: 1888, 510: 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 531: 1888, 1888, 720: 3230, 3228}, - {1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 460: 1889, 1889, 465: 1889, 1889, 468: 1889, 1889, 1889, 474: 1889, 476: 1889, 1889, 479: 1889, 1889, 1889, 483: 1889, 485: 1889, 487: 1889, 1889, 1889, 1889, 3234, 494: 1889, 3233, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 508: 1889, 510: 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 531: 1889, 1889, 720: 3230, 3228}, - {1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 460: 1890, 1890, 465: 1890, 1890, 468: 1890, 1890, 1890, 474: 1890, 476: 1890, 1890, 479: 1890, 1890, 1890, 483: 1890, 485: 1890, 487: 1890, 1890, 1890, 1890, 3234, 494: 1890, 3233, 1890, 3229, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 508: 1890, 510: 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 531: 1890, 1890, 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, 3382}, - {57: 3383, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, + {1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 460: 1888, 1888, 465: 1888, 1888, 468: 1888, 1888, 1888, 474: 1888, 476: 1888, 1888, 479: 1888, 1888, 1888, 483: 1888, 485: 1888, 487: 1888, 1888, 1888, 1888, 492: 1888, 494: 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 508: 1888, 510: 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 1888, 531: 1888, 1888, 720: 3231, 3229}, + {1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 460: 1889, 1889, 465: 1889, 1889, 468: 1889, 1889, 1889, 474: 1889, 476: 1889, 1889, 479: 1889, 1889, 1889, 483: 1889, 485: 1889, 487: 1889, 1889, 1889, 1889, 492: 3235, 494: 1889, 3234, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 508: 1889, 510: 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 1889, 531: 1889, 1889, 720: 3231, 3229}, + {1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 460: 1890, 1890, 465: 1890, 1890, 468: 1890, 1890, 1890, 474: 1890, 476: 1890, 1890, 479: 1890, 1890, 1890, 483: 1890, 485: 1890, 487: 1890, 1890, 1890, 1890, 492: 3235, 494: 1890, 3234, 1890, 3230, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 508: 1890, 510: 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 1890, 531: 1890, 1890, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3383}, + {57: 3384, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, // 935 - {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}, + {144: 3247, 474: 3248, 727: 903, 867: 3385}, + {727: 3251, 734: 3386}, + {910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, 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, 492: 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: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3388}, + {57: 3389, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, // 940 - {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, 1072: 3393}, + {144: 3247, 474: 3248, 727: 903, 867: 3390}, + {727: 3251, 734: 3391}, + {911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 911, 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, 492: 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: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3393}, + {7: 3395, 57: 908, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229, 1073: 3394}, // 945 - {57: 3401}, - {493: 3309, 559: 3311, 3310, 564: 3396, 805: 3395}, - {7: 3398, 57: 905, 1073: 3400}, - {7: 3398, 57: 905, 1073: 3397}, + {57: 3402}, + {493: 3310, 559: 3312, 3311, 564: 3397, 806: 3396}, + {7: 3399, 57: 905, 1074: 3401}, + {7: 3399, 57: 905, 1074: 3398}, {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3400}, + {57: 904, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, {57: 907}, - {144: 3246, 474: 3247, 727: 903, 864: 3402}, - {727: 3250, 734: 3403}, + {144: 3247, 474: 3248, 727: 903, 867: 3403}, + {727: 3251, 734: 3404}, // 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, 1072: 3406}, - {57: 3407}, - {144: 3246, 474: 3247, 727: 903, 864: 3408}, + {912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 912, 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, 492: 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: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3406}, + {7: 3395, 57: 908, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229, 1073: 3407}, + {57: 3408}, + {144: 3247, 474: 3248, 727: 903, 867: 3409}, // 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}, - {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, 3411, 3140, 3223, 3139, 3136}, - {57: 3412, 467: 3238, 570: 3239}, - {727: 3250, 734: 3413}, + {727: 3251, 734: 3410}, + {913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 913, 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, 492: 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3412, 3141, 3224, 3140, 3137}, + {57: 3413, 467: 3239, 570: 3240}, + {727: 3251, 734: 3414}, // 965 - {914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 460: 914, 914, 914, 914, 465: 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 476: 914, 914, 479: 914, 914, 914, 914, 914, 485: 914, 487: 914, 914, 914, 914, 914, 494: 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 530: 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 570: 914}, - {57: 3415}, - {727: 3250, 734: 3416}, - {915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 460: 915, 915, 915, 915, 465: 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 476: 915, 915, 479: 915, 915, 915, 915, 915, 485: 915, 487: 915, 915, 915, 915, 915, 494: 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 530: 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 570: 915}, - {57: 3418}, + {914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 460: 914, 914, 914, 914, 465: 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 476: 914, 914, 479: 914, 914, 914, 914, 914, 485: 914, 487: 914, 914, 914, 914, 492: 914, 494: 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 530: 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 914, 570: 914}, + {57: 3416}, + {727: 3251, 734: 3417}, + {915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 460: 915, 915, 915, 915, 465: 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 476: 915, 915, 479: 915, 915, 915, 915, 915, 485: 915, 487: 915, 915, 915, 915, 492: 915, 494: 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 530: 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 915, 570: 915}, + {57: 3419}, // 970 - {727: 3250, 734: 3419}, - {916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 460: 916, 916, 916, 916, 465: 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 476: 916, 916, 479: 916, 916, 916, 916, 916, 485: 916, 487: 916, 916, 916, 916, 916, 494: 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 530: 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 570: 916}, - {57: 3421}, - {727: 3250, 734: 3422}, - {917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 460: 917, 917, 917, 917, 465: 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 476: 917, 917, 479: 917, 917, 917, 917, 917, 485: 917, 487: 917, 917, 917, 917, 917, 494: 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 530: 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 570: 917}, + {727: 3251, 734: 3420}, + {916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 460: 916, 916, 916, 916, 465: 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 476: 916, 916, 479: 916, 916, 916, 916, 916, 485: 916, 487: 916, 916, 916, 916, 492: 916, 494: 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 530: 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 570: 916}, + {57: 3422}, + {727: 3251, 734: 3423}, + {917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 460: 917, 917, 917, 917, 465: 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 476: 917, 917, 479: 917, 917, 917, 917, 917, 485: 917, 487: 917, 917, 917, 917, 492: 917, 494: 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 530: 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 570: 917}, // 975 - {57: 3424}, - {727: 3250, 734: 3425}, - {918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 460: 918, 918, 918, 918, 465: 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 476: 918, 918, 479: 918, 918, 918, 918, 918, 485: 918, 487: 918, 918, 918, 918, 918, 494: 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 530: 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 570: 918}, - {57: 3427}, - {727: 3250, 734: 3428}, + {57: 3425}, + {727: 3251, 734: 3426}, + {918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 460: 918, 918, 918, 918, 465: 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 476: 918, 918, 479: 918, 918, 918, 918, 918, 485: 918, 487: 918, 918, 918, 918, 492: 918, 494: 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 530: 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 570: 918}, + {57: 3428}, + {727: 3251, 734: 3429}, // 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, 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}, + {919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 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, 492: 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, 491: 1203, 493: 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: 3433, 732: 3431, 3432, 768: 3434, 774: 3435, 800: 3437, 802: 3436}, + {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, 491: 1207, 493: 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, 746: 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, 491: 1206, 493: 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, 746: 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, 491: 1205, 493: 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, 746: 1205, 755: 1205, 1205, 1205}, // 985 - {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, 529: 1204, 552: 1204, 554: 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, 636: 1204, 638: 3441}, - {2: 1202, 1202, 1202, 1202, 1202, 8: 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 58: 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 459: 1202, 461: 1202, 1202, 1202, 1202, 469: 1202, 1202, 1202, 1202, 1202, 478: 1202, 484: 1202, 486: 1202, 492: 1202, 1202, 529: 1202, 552: 1202, 554: 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 564: 1202, 1202, 1202, 1202, 1202, 1202, 572: 1202, 574: 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 636: 1202}, - {2: 1199, 1199, 1199, 1199, 1199, 8: 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 58: 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 459: 1199, 461: 1199, 1199, 1199, 1199, 469: 1199, 1199, 1199, 1199, 1199, 478: 1199, 484: 1199, 486: 1199, 492: 1199, 1199, 529: 1199, 552: 1199, 554: 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 564: 1199, 1199, 1199, 1199, 1199, 1199, 572: 1199, 574: 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 636: 1199}, - {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, 3437}, - {57: 3438, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, + {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, 491: 1204, 493: 1204, 529: 1204, 552: 1204, 554: 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, 636: 1204, 638: 3442}, + {2: 1202, 1202, 1202, 1202, 1202, 8: 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 58: 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 459: 1202, 461: 1202, 1202, 1202, 1202, 469: 1202, 1202, 1202, 1202, 1202, 478: 1202, 484: 1202, 486: 1202, 491: 1202, 493: 1202, 529: 1202, 552: 1202, 554: 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 564: 1202, 1202, 1202, 1202, 1202, 1202, 572: 1202, 574: 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 1202, 636: 1202}, + {2: 1199, 1199, 1199, 1199, 1199, 8: 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 58: 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 459: 1199, 461: 1199, 1199, 1199, 1199, 469: 1199, 1199, 1199, 1199, 1199, 478: 1199, 484: 1199, 486: 1199, 491: 1199, 493: 1199, 529: 1199, 552: 1199, 554: 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 564: 1199, 1199, 1199, 1199, 1199, 1199, 572: 1199, 574: 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 1199, 636: 1199}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3438}, + {57: 3439, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, // 990 - {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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: 3439}, - {1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 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, 771: 3434, 800: 3443, 802: 3435}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3440}, + {1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 1086, 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, 492: 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, 492: 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, 491: 1198, 493: 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, 491: 1203, 493: 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: 3433, 732: 3431, 3432, 768: 3434, 774: 3435, 800: 3444, 802: 3436}, // 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, 771: 3434, 800: 3448, 802: 3435}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3445}, + {57: 3446, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3447}, + {1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 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, 492: 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, 491: 1203, 493: 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: 3433, 732: 3431, 3432, 768: 3434, 774: 3435, 800: 3449, 802: 3436}, // 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, 771: 3434, 800: 3453, 802: 3435}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3450}, + {57: 3451, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3452}, + {1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 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, 492: 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, 491: 1203, 493: 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: 3433, 732: 3431, 3432, 768: 3434, 774: 3435, 800: 3454, 802: 3436}, // 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, 771: 3434, 800: 3458, 802: 3435}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3455}, + {57: 3456, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3457}, + {1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 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, 492: 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, 491: 1203, 493: 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: 3433, 732: 3431, 3432, 768: 3434, 774: 3435, 800: 3459, 802: 3436}, // 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, 771: 3434, 800: 3463, 802: 3435}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3460}, + {57: 3461, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3462}, + {1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 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, 492: 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, 491: 1203, 493: 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: 3433, 732: 3431, 3432, 768: 3434, 774: 3435, 800: 3464, 802: 3436}, // 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, 771: 3434, 800: 3468, 802: 3435}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3465}, + {57: 3466, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3467}, + {1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 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, 492: 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, 491: 1203, 493: 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: 3433, 732: 3431, 3432, 768: 3434, 774: 3435, 800: 3469, 802: 3436}, // 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, 771: 3434, 800: 3473, 802: 3435}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3470}, + {57: 3471, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3472}, + {1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 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, 492: 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, 491: 1203, 493: 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: 3433, 732: 3431, 3432, 768: 3434, 774: 3435, 800: 3474, 802: 3436}, // 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}, - {7: 3476, 57: 1258, 124: 1258, 490: 2627, 753: 2628, 797: 3477}, - {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, 3483}, - {57: 1079, 124: 3479, 1238: 3478}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 761: 3476}, + {1870, 1870, 7: 1870, 57: 1870, 124: 1870, 468: 1870, 490: 1870, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {7: 3477, 57: 1258, 124: 1258, 490: 2628, 753: 2629, 797: 3478}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3484}, + {57: 1079, 124: 3480, 1238: 3479}, // 1030 - {57: 3481}, - {461: 3480}, + {57: 3482}, + {461: 3481}, {57: 1078}, - {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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: 3482}, - {1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 460: 1093, 1093, 1093, 1093, 465: 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 476: 1093, 1093, 479: 1093, 1093, 1093, 1093, 1093, 485: 1093, 487: 1093, 1093, 1093, 1093, 1093, 494: 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 530: 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 570: 1093}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3483}, + {1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 460: 1093, 1093, 1093, 1093, 465: 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 476: 1093, 1093, 479: 1093, 1093, 1093, 1093, 1093, 485: 1093, 487: 1093, 1093, 1093, 1093, 492: 1093, 494: 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 530: 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 570: 1093}, // 1035 - {1869, 1869, 7: 1869, 57: 1869, 124: 1869, 468: 1869, 490: 1869, 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, 509: 3488, 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, 638: 3487, 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, 3485, 732: 3430, 3431, 768: 3486}, - {57: 3496, 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, 3474, 761: 3494}, - {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, 3491}, + {1869, 1869, 7: 1869, 57: 1869, 124: 1869, 468: 1869, 490: 1869, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 509: 3489, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 638: 3488, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3486, 732: 3431, 3432, 768: 3487}, + {57: 3497, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 761: 3495}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3492}, // 1040 - {57: 3489}, - {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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: 3490}, - {1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 460: 1094, 1094, 1094, 1094, 465: 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 476: 1094, 1094, 479: 1094, 1094, 1094, 1094, 1094, 485: 1094, 487: 1094, 1094, 1094, 1094, 1094, 494: 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 530: 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 570: 1094}, - {57: 3492, 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: 3493}, + {57: 3490}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3491}, + {1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 460: 1094, 1094, 1094, 1094, 465: 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 476: 1094, 1094, 479: 1094, 1094, 1094, 1094, 1094, 485: 1094, 487: 1094, 1094, 1094, 1094, 492: 1094, 494: 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 530: 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 570: 1094}, + {57: 3493, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3494}, // 1045 - {1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 460: 1096, 1096, 1096, 1096, 465: 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 476: 1096, 1096, 479: 1096, 1096, 1096, 1096, 1096, 485: 1096, 487: 1096, 1096, 1096, 1096, 1096, 494: 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 530: 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 570: 1096}, - {7: 3476, 57: 3495}, - {1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 460: 1097, 1097, 1097, 1097, 465: 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 476: 1097, 1097, 479: 1097, 1097, 1097, 1097, 1097, 485: 1097, 487: 1097, 1097, 1097, 1097, 1097, 494: 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 530: 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 570: 1097}, - {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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: 3497}, - {1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 460: 1095, 1095, 1095, 1095, 465: 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 476: 1095, 1095, 479: 1095, 1095, 1095, 1095, 1095, 485: 1095, 487: 1095, 1095, 1095, 1095, 1095, 494: 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 530: 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 570: 1095}, + {1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 460: 1096, 1096, 1096, 1096, 465: 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 476: 1096, 1096, 479: 1096, 1096, 1096, 1096, 1096, 485: 1096, 487: 1096, 1096, 1096, 1096, 492: 1096, 494: 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 530: 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 570: 1096}, + {7: 3477, 57: 3496}, + {1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 460: 1097, 1097, 1097, 1097, 465: 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 476: 1097, 1097, 479: 1097, 1097, 1097, 1097, 1097, 485: 1097, 487: 1097, 1097, 1097, 1097, 492: 1097, 494: 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 530: 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 570: 1097}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3498}, + {1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 460: 1095, 1095, 1095, 1095, 465: 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 476: 1095, 1095, 479: 1095, 1095, 1095, 1095, 1095, 485: 1095, 487: 1095, 1095, 1095, 1095, 492: 1095, 494: 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 530: 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 570: 1095}, // 1050 - {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, 638: 3500, 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, 3499}, - {57: 3504, 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, 3501}, - {57: 3502, 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: 3503}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 638: 3501, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3500}, + {57: 3505, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3502}, + {57: 3503, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3504}, // 1055 - {1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 460: 1098, 1098, 1098, 1098, 465: 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 476: 1098, 1098, 479: 1098, 1098, 1098, 1098, 1098, 485: 1098, 487: 1098, 1098, 1098, 1098, 1098, 494: 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 530: 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 570: 1098}, - {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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: 3505}, - {1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 460: 1099, 1099, 1099, 1099, 465: 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 476: 1099, 1099, 479: 1099, 1099, 1099, 1099, 1099, 485: 1099, 487: 1099, 1099, 1099, 1099, 1099, 494: 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 530: 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 570: 1099}, - {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, 638: 3508, 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, 3507}, - {57: 3512, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, + {1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 460: 1098, 1098, 1098, 1098, 465: 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 476: 1098, 1098, 479: 1098, 1098, 1098, 1098, 1098, 485: 1098, 487: 1098, 1098, 1098, 1098, 492: 1098, 494: 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 530: 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 570: 1098}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3506}, + {1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 460: 1099, 1099, 1099, 1099, 465: 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 476: 1099, 1099, 479: 1099, 1099, 1099, 1099, 1099, 485: 1099, 487: 1099, 1099, 1099, 1099, 492: 1099, 494: 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 530: 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 570: 1099}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 638: 3509, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3508}, + {57: 3513, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, // 1060 - {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, 3509}, - {57: 3510, 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: 3511}, - {1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 460: 1100, 1100, 1100, 1100, 465: 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 476: 1100, 1100, 479: 1100, 1100, 1100, 1100, 1100, 485: 1100, 487: 1100, 1100, 1100, 1100, 1100, 494: 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 530: 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 570: 1100}, - {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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: 3513}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3510}, + {57: 3511, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3512}, + {1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 460: 1100, 1100, 1100, 1100, 465: 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 476: 1100, 1100, 479: 1100, 1100, 1100, 1100, 1100, 485: 1100, 487: 1100, 1100, 1100, 1100, 492: 1100, 494: 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 530: 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 570: 1100}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3514}, // 1065 - {1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 460: 1101, 1101, 1101, 1101, 465: 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 476: 1101, 1101, 479: 1101, 1101, 1101, 1101, 1101, 485: 1101, 487: 1101, 1101, 1101, 1101, 1101, 494: 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 530: 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 570: 1101}, - {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, 638: 3516, 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, 3515}, - {57: 3520, 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, 3517}, - {57: 3518, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, + {1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 460: 1101, 1101, 1101, 1101, 465: 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 476: 1101, 1101, 479: 1101, 1101, 1101, 1101, 1101, 485: 1101, 487: 1101, 1101, 1101, 1101, 492: 1101, 494: 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 530: 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 570: 1101}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 638: 3517, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3516}, + {57: 3521, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3518}, + {57: 3519, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, // 1070 - {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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: 3519}, - {1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 460: 1102, 1102, 1102, 1102, 465: 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 476: 1102, 1102, 479: 1102, 1102, 1102, 1102, 1102, 485: 1102, 487: 1102, 1102, 1102, 1102, 1102, 494: 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 530: 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 570: 1102}, - {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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: 3521}, - {1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 460: 1103, 1103, 1103, 1103, 465: 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 476: 1103, 1103, 479: 1103, 1103, 1103, 1103, 1103, 485: 1103, 487: 1103, 1103, 1103, 1103, 1103, 494: 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 530: 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 570: 1103}, - {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: 3523}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3520}, + {1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 460: 1102, 1102, 1102, 1102, 465: 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 476: 1102, 1102, 479: 1102, 1102, 1102, 1102, 1102, 485: 1102, 487: 1102, 1102, 1102, 1102, 492: 1102, 494: 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 530: 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 570: 1102}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3522}, + {1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 460: 1103, 1103, 1103, 1103, 465: 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 476: 1103, 1103, 479: 1103, 1103, 1103, 1103, 1103, 485: 1103, 487: 1103, 1103, 1103, 1103, 492: 1103, 494: 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 530: 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 570: 1103}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 761: 3524}, // 1075 - {7: 3476, 57: 3524}, - {1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 460: 1104, 1104, 1104, 1104, 465: 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 476: 1104, 1104, 479: 1104, 1104, 1104, 1104, 1104, 485: 1104, 487: 1104, 1104, 1104, 1104, 1104, 494: 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 530: 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 570: 1104}, - {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: 3526}, - {7: 3476, 57: 3527}, - {1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 460: 1105, 1105, 1105, 1105, 465: 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 476: 1105, 1105, 479: 1105, 1105, 1105, 1105, 1105, 485: 1105, 487: 1105, 1105, 1105, 1105, 1105, 494: 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 530: 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 570: 1105}, + {7: 3477, 57: 3525}, + {1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 460: 1104, 1104, 1104, 1104, 465: 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 476: 1104, 1104, 479: 1104, 1104, 1104, 1104, 1104, 485: 1104, 487: 1104, 1104, 1104, 1104, 492: 1104, 494: 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 530: 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 570: 1104}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 761: 3527}, + {7: 3477, 57: 3528}, + {1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 460: 1105, 1105, 1105, 1105, 465: 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 476: 1105, 1105, 479: 1105, 1105, 1105, 1105, 1105, 485: 1105, 487: 1105, 1105, 1105, 1105, 492: 1105, 494: 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 530: 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 570: 1105}, // 1080 - {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, 3529}, - {7: 3530, 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, 3531}, - {7: 3532, 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, 3533}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3530}, + {7: 3531, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3532}, + {7: 3533, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3534}, // 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, 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}, + {57: 3535, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 1121, 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, 492: 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: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3537, 1159: 3539, 1214: 3540, 1298: 3541, 3538}, + {57: 3549, 487: 3550, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 3543, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3542}, // 1090 - {2: 1112, 1112, 1112, 1112, 1112, 8: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 58: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 459: 1112, 461: 1112, 1112, 1112, 1112, 469: 1112, 1112, 1112, 1112, 1112, 478: 1112, 484: 1112, 486: 1112, 1112, 492: 1112, 1112, 529: 1112, 552: 1112, 554: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 564: 1112, 1112, 1112, 1112, 1112, 1112, 572: 1112, 574: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 636: 1112}, - {2: 1111, 1111, 1111, 1111, 1111, 8: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 58: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 459: 1111, 461: 1111, 1111, 1111, 1111, 469: 1111, 1111, 1111, 1111, 1111, 478: 1111, 484: 1111, 486: 1111, 1111, 492: 1111, 1111, 529: 1111, 552: 1111, 554: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 564: 1111, 1111, 1111, 1111, 1111, 1111, 572: 1111, 574: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 636: 1111}, - {2: 1110, 1110, 1110, 1110, 1110, 8: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 58: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 459: 1110, 461: 1110, 1110, 1110, 1110, 469: 1110, 1110, 1110, 1110, 1110, 478: 1110, 484: 1110, 486: 1110, 1110, 492: 1110, 1110, 529: 1110, 552: 1110, 554: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 564: 1110, 1110, 1110, 1110, 1110, 1110, 572: 1110, 574: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 636: 1110}, - {487: 3545, 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, 3543}, + {2: 1112, 1112, 1112, 1112, 1112, 8: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 58: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 459: 1112, 461: 1112, 1112, 1112, 1112, 469: 1112, 1112, 1112, 1112, 1112, 478: 1112, 484: 1112, 486: 1112, 1112, 491: 1112, 493: 1112, 529: 1112, 552: 1112, 554: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 564: 1112, 1112, 1112, 1112, 1112, 1112, 572: 1112, 574: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 636: 1112}, + {2: 1111, 1111, 1111, 1111, 1111, 8: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 58: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 459: 1111, 461: 1111, 1111, 1111, 1111, 469: 1111, 1111, 1111, 1111, 1111, 478: 1111, 484: 1111, 486: 1111, 1111, 491: 1111, 493: 1111, 529: 1111, 552: 1111, 554: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 564: 1111, 1111, 1111, 1111, 1111, 1111, 572: 1111, 574: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 636: 1111}, + {2: 1110, 1110, 1110, 1110, 1110, 8: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 58: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 459: 1110, 461: 1110, 1110, 1110, 1110, 469: 1110, 1110, 1110, 1110, 1110, 478: 1110, 484: 1110, 486: 1110, 1110, 491: 1110, 493: 1110, 529: 1110, 552: 1110, 554: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 564: 1110, 1110, 1110, 1110, 1110, 1110, 572: 1110, 574: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 636: 1110}, + {487: 3546, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3544}, // 1095 - {57: 3544, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 460: 1127, 1127, 1127, 1127, 465: 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 476: 1127, 1127, 479: 1127, 1127, 1127, 1127, 1127, 485: 1127, 487: 1127, 1127, 1127, 1127, 1127, 494: 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 530: 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 570: 1127}, - {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, 3546}, - {57: 3547, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 460: 1126, 1126, 1126, 1126, 465: 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 476: 1126, 1126, 479: 1126, 1126, 1126, 1126, 1126, 485: 1126, 487: 1126, 1126, 1126, 1126, 1126, 494: 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 530: 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 570: 1126}, + {57: 3545, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 460: 1127, 1127, 1127, 1127, 465: 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 476: 1127, 1127, 479: 1127, 1127, 1127, 1127, 1127, 485: 1127, 487: 1127, 1127, 1127, 1127, 492: 1127, 494: 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 530: 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 1127, 570: 1127}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3547}, + {57: 3548, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 460: 1126, 1126, 1126, 1126, 465: 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 476: 1126, 1126, 479: 1126, 1126, 1126, 1126, 1126, 485: 1126, 487: 1126, 1126, 1126, 1126, 492: 1126, 494: 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 530: 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 1126, 570: 1126}, // 1100 - {1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 460: 1129, 1129, 1129, 1129, 465: 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 476: 1129, 1129, 479: 1129, 1129, 1129, 1129, 1129, 485: 1129, 487: 1129, 1129, 1129, 1129, 1129, 494: 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 530: 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 570: 1129}, - {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, 3550}, - {57: 3551, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 460: 1128, 1128, 1128, 1128, 465: 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 476: 1128, 1128, 479: 1128, 1128, 1128, 1128, 1128, 485: 1128, 487: 1128, 1128, 1128, 1128, 1128, 494: 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 530: 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 570: 1128}, - {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, 3553}, + {1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 460: 1129, 1129, 1129, 1129, 465: 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 476: 1129, 1129, 479: 1129, 1129, 1129, 1129, 1129, 485: 1129, 487: 1129, 1129, 1129, 1129, 492: 1129, 494: 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 530: 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 570: 1129}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3551}, + {57: 3552, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 460: 1128, 1128, 1128, 1128, 465: 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 476: 1128, 1128, 479: 1128, 1128, 1128, 1128, 1128, 485: 1128, 487: 1128, 1128, 1128, 1128, 492: 1128, 494: 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 530: 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 570: 1128}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3554}, // 1105 - {7: 3554, 487: 3555, 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, 3561}, - {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, 3556}, - {57: 3557, 479: 3558, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 460: 1134, 1134, 1134, 1134, 465: 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 476: 1134, 1134, 479: 1134, 1134, 1134, 1134, 1134, 485: 1134, 487: 1134, 1134, 1134, 1134, 1134, 494: 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 530: 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 570: 1134}, + {7: 3555, 487: 3556, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3562}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3557}, + {57: 3558, 479: 3559, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 460: 1134, 1134, 1134, 1134, 465: 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 476: 1134, 1134, 479: 1134, 1134, 1134, 1134, 1134, 485: 1134, 487: 1134, 1134, 1134, 1134, 492: 1134, 494: 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 530: 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 570: 1134}, // 1110 - {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, 3559}, - {57: 3560, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 460: 1132, 1132, 1132, 1132, 465: 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 476: 1132, 1132, 479: 1132, 1132, 1132, 1132, 1132, 485: 1132, 487: 1132, 1132, 1132, 1132, 1132, 494: 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 530: 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 570: 1132}, - {7: 3563, 57: 3562, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 460: 1135, 1135, 1135, 1135, 465: 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 476: 1135, 1135, 479: 1135, 1135, 1135, 1135, 1135, 485: 1135, 487: 1135, 1135, 1135, 1135, 1135, 494: 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 530: 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 570: 1135}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3560}, + {57: 3561, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 460: 1132, 1132, 1132, 1132, 465: 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 476: 1132, 1132, 479: 1132, 1132, 1132, 1132, 1132, 485: 1132, 487: 1132, 1132, 1132, 1132, 492: 1132, 494: 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 530: 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 570: 1132}, + {7: 3564, 57: 3563, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 460: 1135, 1135, 1135, 1135, 465: 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 476: 1135, 1135, 479: 1135, 1135, 1135, 1135, 1135, 485: 1135, 487: 1135, 1135, 1135, 1135, 492: 1135, 494: 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 530: 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 570: 1135}, // 1115 - {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, 3564}, - {57: 3565, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 460: 1133, 1133, 1133, 1133, 465: 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 476: 1133, 1133, 479: 1133, 1133, 1133, 1133, 1133, 485: 1133, 487: 1133, 1133, 1133, 1133, 1133, 494: 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 530: 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 570: 1133}, - {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, 3567}, - {471: 3573, 3572, 3578, 509: 3574, 530: 3580, 542: 3575, 3576, 3569, 3579, 3568, 3577, 3570, 3571}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3565}, + {57: 3566, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 460: 1133, 1133, 1133, 1133, 465: 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 476: 1133, 1133, 479: 1133, 1133, 1133, 1133, 1133, 485: 1133, 487: 1133, 1133, 1133, 1133, 492: 1133, 494: 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 530: 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 570: 1133}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3568}, + {471: 3574, 3573, 3579, 509: 3575, 530: 3581, 542: 3576, 3577, 3570, 3580, 3569, 3578, 3571, 3572}, // 1120 - {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, 3600}, - {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, 3599}, - {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, 3598}, - {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, 3597}, - {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, 3594, 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, 3593}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3601}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3600}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3599}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3598}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3595, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3594}, // 1125 - {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, 3590, 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, 3589}, - {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, 3588}, - {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, 3587}, - {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, 3586}, - {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, 3585}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3591, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3590}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3589}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3588}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3587}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3586}, // 1130 - {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, 3584}, - {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, 3583}, - {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, 3581}, - {57: 3582, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 460: 1136, 1136, 1136, 1136, 465: 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 476: 1136, 1136, 479: 1136, 1136, 1136, 1136, 1136, 485: 1136, 487: 1136, 1136, 1136, 1136, 1136, 494: 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 530: 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 570: 1136}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3585}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3584}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3582}, + {57: 3583, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 460: 1136, 1136, 1136, 1136, 465: 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 476: 1136, 1136, 479: 1136, 1136, 1136, 1136, 1136, 485: 1136, 487: 1136, 1136, 1136, 1136, 492: 1136, 494: 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 530: 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 570: 1136}, // 1135 - {1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 460: 1243, 1243, 1243, 1243, 465: 1243, 1243, 468: 1243, 1243, 1243, 1243, 1243, 1243, 1243, 476: 1243, 1243, 479: 1243, 1243, 1243, 1243, 1243, 485: 1243, 487: 1243, 1243, 1243, 1243, 1243, 494: 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 530: 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243}, - {1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 460: 1244, 1244, 1244, 1244, 465: 1244, 1244, 468: 1244, 1244, 1244, 1244, 1244, 1244, 1244, 476: 1244, 1244, 479: 1244, 1244, 1244, 1244, 1244, 485: 1244, 487: 1244, 1244, 1244, 1244, 1244, 494: 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 530: 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 3579, 1244, 1244, 1244, 1244, 1244, 1244}, - {1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 460: 1245, 1245, 1245, 1245, 465: 1245, 1245, 468: 1245, 1245, 1245, 1245, 1245, 1245, 1245, 476: 1245, 1245, 479: 1245, 1245, 1245, 1245, 1245, 485: 1245, 487: 1245, 1245, 1245, 1245, 1245, 494: 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 530: 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 3579, 1245, 1245, 1245, 1245, 1245, 1245}, - {1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 460: 1246, 1246, 1246, 1246, 465: 1246, 1246, 468: 1246, 1246, 1246, 1246, 1246, 1246, 1246, 476: 1246, 1246, 479: 1246, 1246, 1246, 1246, 1246, 485: 1246, 487: 1246, 1246, 1246, 1246, 1246, 494: 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 530: 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 3579, 1246, 1246, 1246, 1246, 1246, 1246}, - {1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 460: 1247, 1247, 1247, 1247, 465: 1247, 1247, 468: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 476: 1247, 1247, 479: 1247, 1247, 1247, 1247, 1247, 485: 1247, 487: 1247, 1247, 1247, 1247, 1247, 494: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 530: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 3579, 1247, 1247, 1247, 1247, 1247, 1247}, + {1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 460: 1243, 1243, 1243, 1243, 465: 1243, 1243, 468: 1243, 1243, 1243, 1243, 1243, 1243, 1243, 476: 1243, 1243, 479: 1243, 1243, 1243, 1243, 1243, 485: 1243, 487: 1243, 1243, 1243, 1243, 492: 1243, 494: 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 530: 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243}, + {1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 460: 1244, 1244, 1244, 1244, 465: 1244, 1244, 468: 1244, 1244, 1244, 1244, 1244, 1244, 1244, 476: 1244, 1244, 479: 1244, 1244, 1244, 1244, 1244, 485: 1244, 487: 1244, 1244, 1244, 1244, 492: 1244, 494: 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 530: 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 3580, 1244, 1244, 1244, 1244, 1244, 1244}, + {1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 460: 1245, 1245, 1245, 1245, 465: 1245, 1245, 468: 1245, 1245, 1245, 1245, 1245, 1245, 1245, 476: 1245, 1245, 479: 1245, 1245, 1245, 1245, 1245, 485: 1245, 487: 1245, 1245, 1245, 1245, 492: 1245, 494: 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 530: 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 3580, 1245, 1245, 1245, 1245, 1245, 1245}, + {1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 460: 1246, 1246, 1246, 1246, 465: 1246, 1246, 468: 1246, 1246, 1246, 1246, 1246, 1246, 1246, 476: 1246, 1246, 479: 1246, 1246, 1246, 1246, 1246, 485: 1246, 487: 1246, 1246, 1246, 1246, 492: 1246, 494: 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 530: 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 3580, 1246, 1246, 1246, 1246, 1246, 1246}, + {1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 460: 1247, 1247, 1247, 1247, 465: 1247, 1247, 468: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 476: 1247, 1247, 479: 1247, 1247, 1247, 1247, 1247, 485: 1247, 487: 1247, 1247, 1247, 1247, 492: 1247, 494: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 530: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 3580, 1247, 1247, 1247, 1247, 1247, 1247}, // 1140 - {1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 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, 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}, + {1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 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, 492: 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, 3580, 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, 3579, 1251, 476: 1251, 1251, 479: 1251, 1251, 1251, 1251, 1251, 485: 1251, 487: 1251, 1251, 1251, 1251, 492: 1251, 494: 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 3575, 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, 3576, 3577, 1251, 3580, 1251, 3578, 1251, 1251, 1251, 1251}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3592}, + {105: 3351, 3347, 108: 3344, 3359, 111: 3346, 3343, 3345, 3349, 3350, 3355, 3354, 3353, 3357, 3358, 3352, 3356, 3348, 492: 3235, 494: 3233, 3234, 3232, 3230, 517: 3341, 3338, 3340, 3339, 3335, 3337, 3336, 3333, 3334, 3332, 3342, 720: 3231, 3229, 793: 3331, 816: 3593}, + {1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 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, 492: 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, 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}, + {1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 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, 3579, 1252, 476: 1252, 1252, 479: 1252, 1252, 1252, 1252, 1252, 485: 1252, 487: 1252, 1252, 1252, 1252, 492: 1252, 494: 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 3575, 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, 3576, 3577, 1252, 3580, 1252, 3578, 1252, 1252, 1252, 1252}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3596}, + {105: 3351, 3347, 108: 3344, 3359, 111: 3346, 3343, 3345, 3349, 3350, 3355, 3354, 3353, 3357, 3358, 3352, 3356, 3348, 492: 3235, 494: 3233, 3234, 3232, 3230, 517: 3341, 3338, 3340, 3339, 3335, 3337, 3336, 3333, 3334, 3332, 3342, 720: 3231, 3229, 793: 3331, 816: 3597}, + {1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 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, 492: 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, 3574, 3573, 3579, 1253, 476: 1253, 1253, 479: 1253, 1253, 1253, 1253, 1253, 485: 1253, 487: 1253, 1253, 1253, 1253, 492: 1253, 494: 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 3575, 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, 3576, 3577, 1253, 3580, 1253, 3578, 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, 792: 3330, 816: 3602}, - {487: 3603}, + {1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 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, 3574, 3573, 3579, 1254, 476: 1254, 1254, 479: 1254, 1254, 1254, 1254, 1254, 485: 1254, 487: 1254, 1254, 1254, 1254, 492: 1254, 494: 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 3575, 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, 3576, 3577, 1254, 3580, 1254, 3578, 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, 3574, 3573, 3579, 1255, 476: 1255, 1255, 479: 1255, 1255, 1255, 1255, 1255, 485: 1255, 487: 1255, 1255, 1255, 1255, 492: 1255, 494: 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 3575, 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, 3576, 3577, 1255, 3580, 1255, 3578, 3571, 3572, 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, 3574, 3573, 3579, 1256, 476: 1256, 1256, 479: 1256, 1256, 1256, 1256, 1256, 485: 1256, 487: 1256, 1256, 1256, 1256, 492: 1256, 494: 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 3575, 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, 3576, 3577, 3570, 3580, 1256, 3578, 3571, 3572, 1256, 1256}, + {105: 3351, 3347, 108: 3344, 3359, 111: 3346, 3343, 3345, 3349, 3350, 3355, 3354, 3353, 3357, 3358, 3352, 3356, 3348, 517: 3341, 3338, 3340, 3339, 3335, 3337, 3336, 3333, 3334, 3332, 3342, 793: 3331, 816: 3603}, + {487: 3604}, // 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}, - {57: 3605, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 460: 1138, 1138, 1138, 1138, 465: 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 476: 1138, 1138, 479: 1138, 1138, 1138, 1138, 1138, 485: 1138, 487: 1138, 1138, 1138, 1138, 1138, 494: 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 530: 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 570: 1138}, - {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, 3607}, - {7: 3608, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3605}, + {57: 3606, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 460: 1138, 1138, 1138, 1138, 465: 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 476: 1138, 1138, 479: 1138, 1138, 1138, 1138, 1138, 485: 1138, 487: 1138, 1138, 1138, 1138, 492: 1138, 494: 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 530: 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 570: 1138}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3608}, + {7: 3609, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, // 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, 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}, + {567: 3610}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3611}, + {105: 3351, 3347, 108: 3344, 3359, 111: 3346, 3343, 3345, 3349, 3350, 3355, 3354, 3353, 3357, 3358, 3352, 3356, 3348, 492: 3235, 494: 3233, 3234, 3232, 3230, 517: 3341, 3338, 3340, 3339, 3335, 3337, 3336, 3333, 3334, 3332, 3342, 720: 3231, 3229, 793: 3331, 816: 3612}, + {57: 3613}, + {1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 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, 492: 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 - {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, 3614}, - {7: 3615, 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, 3617, 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, 3616}, - {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3615}, + {7: 3616, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3618, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3617}, + {57: 3622, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3619}, // 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, 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, 1031: 3623, 3625}, + {105: 3351, 3347, 108: 3344, 3359, 111: 3346, 3343, 3345, 3349, 3350, 3355, 3354, 3353, 3357, 3358, 3352, 3356, 3348, 492: 3235, 494: 3233, 3234, 3232, 3230, 517: 3341, 3338, 3340, 3339, 3335, 3337, 3336, 3333, 3334, 3332, 3342, 720: 3231, 3229, 793: 3331, 816: 3620}, + {57: 3621}, + {1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 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, 492: 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, 492: 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: 3625, 1032: 3624, 3626}, // 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, 1031: 3623, 3628}, + {57: 3627}, + {1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 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, 492: 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: 3625, 1032: 3624, 3629}, // 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}, - {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, 3631}, - {7: 3632, 471: 3573, 3572, 3578, 509: 3574, 542: 3575, 3576, 3569, 3579, 3568, 3577, 3570, 3571}, - {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, 3633}, + {57: 3630}, + {1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 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, 492: 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3632}, + {7: 3633, 471: 3574, 3573, 3579, 509: 3575, 542: 3576, 3577, 3570, 3580, 3569, 3578, 3571, 3572}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3634}, // 1185 - {57: 3634, 471: 3573, 3572, 3578, 509: 3574, 542: 3575, 3576, 3569, 3579, 3568, 3577, 3570, 3571}, - {1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 460: 1145, 1145, 1145, 1145, 465: 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 476: 1145, 1145, 479: 1145, 1145, 1145, 1145, 1145, 485: 1145, 487: 1145, 1145, 1145, 1145, 1145, 494: 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 530: 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 570: 1145}, - {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, 1866, 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: 3636, 811: 3637}, - {7: 3476, 57: 1865}, - {57: 3638}, + {57: 3635, 471: 3574, 3573, 3579, 509: 3575, 542: 3576, 3577, 3570, 3580, 3569, 3578, 3571, 3572}, + {1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 460: 1145, 1145, 1145, 1145, 465: 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 476: 1145, 1145, 479: 1145, 1145, 1145, 1145, 1145, 485: 1145, 487: 1145, 1145, 1145, 1145, 492: 1145, 494: 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 530: 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 570: 1145}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 1866, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 761: 3637, 812: 3638}, + {7: 3477, 57: 1865}, + {57: 3639}, // 1190 - {1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 460: 1146, 1146, 1146, 1146, 465: 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 476: 1146, 1146, 479: 1146, 1146, 1146, 1146, 1146, 485: 1146, 487: 1146, 1146, 1146, 1146, 1146, 494: 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 530: 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 570: 1146}, - {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, 793: 3643}, + {1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 460: 1146, 1146, 1146, 1146, 465: 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 476: 1146, 1146, 479: 1146, 1146, 1146, 1146, 1146, 485: 1146, 487: 1146, 1146, 1146, 1146, 492: 1146, 494: 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 530: 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 570: 1146}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 761: 3641}, + {7: 3477, 57: 3642, 468: 3643}, + {1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 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, 492: 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: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 529: 3646, 648: 3378, 2665, 2666, 2664, 725: 3645, 794: 3644}, // 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}, - {723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 98: 723, 107: 723, 458: 723, 723, 723, 462: 723, 723, 723, 723, 723, 723, 474: 723, 723, 478: 723, 484: 723, 723, 723, 492: 723, 498: 723, 529: 723, 553: 723, 563: 723, 571: 723, 573: 723, 629: 723, 723, 723, 723, 723, 723, 723, 645: 723}, - {1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 460: 1150, 1150, 1150, 1150, 465: 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 476: 1150, 1150, 479: 1150, 1150, 1150, 1150, 1150, 485: 1150, 487: 1150, 1150, 1150, 1150, 1150, 494: 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 530: 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 570: 1150}, - {1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 460: 1152, 1152, 1152, 1152, 465: 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 476: 1152, 1152, 479: 1152, 1152, 1152, 1152, 1152, 485: 1152, 487: 1152, 1152, 1152, 1152, 1152, 494: 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 530: 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 570: 1152}, + {57: 3647}, + {724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 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, 491: 724, 498: 724, 529: 724, 553: 724, 563: 724, 571: 724, 573: 724, 629: 724, 724, 724, 724, 724, 724, 724, 645: 724}, + {723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 98: 723, 107: 723, 458: 723, 723, 723, 462: 723, 723, 723, 723, 723, 723, 474: 723, 723, 478: 723, 484: 723, 723, 723, 491: 723, 498: 723, 529: 723, 553: 723, 563: 723, 571: 723, 573: 723, 629: 723, 723, 723, 723, 723, 723, 723, 645: 723}, + {1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 460: 1150, 1150, 1150, 1150, 465: 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 476: 1150, 1150, 479: 1150, 1150, 1150, 1150, 1150, 485: 1150, 487: 1150, 1150, 1150, 1150, 492: 1150, 494: 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 530: 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 570: 1150}, + {1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 460: 1152, 1152, 1152, 1152, 465: 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 476: 1152, 1152, 479: 1152, 1152, 1152, 1152, 1152, 485: 1152, 487: 1152, 1152, 1152, 1152, 492: 1152, 494: 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 530: 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 570: 1152}, // 1200 - {57: 3649, 493: 3650}, - {1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 460: 1074, 1074, 1074, 1074, 465: 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 476: 1074, 1074, 479: 1074, 1074, 1074, 1074, 1074, 485: 1074, 487: 1074, 1074, 1074, 1074, 1074, 494: 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 530: 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 570: 1074}, - {57: 3651}, - {1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 460: 1073, 1073, 1073, 1073, 465: 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 476: 1073, 1073, 479: 1073, 1073, 1073, 1073, 1073, 485: 1073, 487: 1073, 1073, 1073, 1073, 1073, 494: 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 530: 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 570: 1073}, - {57: 3653}, + {57: 3650, 493: 3651}, + {1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 460: 1074, 1074, 1074, 1074, 465: 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 476: 1074, 1074, 479: 1074, 1074, 1074, 1074, 1074, 485: 1074, 487: 1074, 1074, 1074, 1074, 492: 1074, 494: 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 530: 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 570: 1074}, + {57: 3652}, + {1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 460: 1073, 1073, 1073, 1073, 465: 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 476: 1073, 1073, 479: 1073, 1073, 1073, 1073, 1073, 485: 1073, 487: 1073, 1073, 1073, 1073, 492: 1073, 494: 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 530: 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 570: 1073}, + {57: 3654}, // 1205 - {1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 460: 1153, 1153, 1153, 1153, 465: 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 476: 1153, 1153, 479: 1153, 1153, 1153, 1153, 1153, 485: 1153, 487: 1153, 1153, 1153, 1153, 1153, 494: 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 530: 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 570: 1153}, - {57: 3656}, - {1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 460: 1154, 1154, 1154, 1154, 465: 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 476: 1154, 1154, 479: 1154, 1154, 1154, 1154, 1154, 485: 1154, 487: 1154, 1154, 1154, 1154, 1154, 494: 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 530: 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 570: 1154}, - {1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 460: 1167, 1167, 1167, 1167, 465: 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 476: 1167, 1167, 479: 1167, 1167, 1167, 1167, 1167, 485: 1167, 487: 1167, 1167, 1167, 1167, 1167, 494: 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 530: 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 570: 1167, 637: 1167, 652: 1167, 655: 1167}, - {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, 1866, 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: 3636, 811: 3658}, + {1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 460: 1153, 1153, 1153, 1153, 465: 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 476: 1153, 1153, 479: 1153, 1153, 1153, 1153, 1153, 485: 1153, 487: 1153, 1153, 1153, 1153, 492: 1153, 494: 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 530: 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 570: 1153}, + {57: 3657}, + {1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 460: 1154, 1154, 1154, 1154, 465: 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 476: 1154, 1154, 479: 1154, 1154, 1154, 1154, 1154, 485: 1154, 487: 1154, 1154, 1154, 1154, 492: 1154, 494: 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 530: 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 570: 1154}, + {1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 460: 1167, 1167, 1167, 1167, 465: 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 476: 1167, 1167, 479: 1167, 1167, 1167, 1167, 1167, 485: 1167, 487: 1167, 1167, 1167, 1167, 492: 1167, 494: 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 530: 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 570: 1167, 637: 1167, 652: 1167, 655: 1167}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 1866, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 761: 3637, 812: 3659}, // 1210 - {57: 3659}, - {1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 460: 1155, 1155, 1155, 1155, 465: 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 476: 1155, 1155, 479: 1155, 1155, 1155, 1155, 1155, 485: 1155, 487: 1155, 1155, 1155, 1155, 1155, 494: 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 530: 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 570: 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, 1866, 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: 3636, 811: 3661}, - {57: 3662}, - {1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 460: 1156, 1156, 1156, 1156, 465: 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 476: 1156, 1156, 479: 1156, 1156, 1156, 1156, 1156, 485: 1156, 487: 1156, 1156, 1156, 1156, 1156, 494: 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 530: 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 570: 1156}, + {57: 3660}, + {1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 460: 1155, 1155, 1155, 1155, 465: 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 476: 1155, 1155, 479: 1155, 1155, 1155, 1155, 1155, 485: 1155, 487: 1155, 1155, 1155, 1155, 492: 1155, 494: 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 530: 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 570: 1155}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 1866, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 761: 3637, 812: 3662}, + {57: 3663}, + {1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 460: 1156, 1156, 1156, 1156, 465: 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 476: 1156, 1156, 479: 1156, 1156, 1156, 1156, 1156, 485: 1156, 487: 1156, 1156, 1156, 1156, 492: 1156, 494: 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 530: 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 570: 1156}, // 1215 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 3665}, - {57: 1241, 482: 1241, 639: 3667}, - {57: 3666}, - {1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 460: 1210, 1210, 1210, 1210, 465: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 476: 1210, 1210, 479: 1210, 1210, 1210, 1210, 1210, 485: 1210, 487: 1210, 1210, 1210, 1210, 1210, 494: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 530: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 570: 1210}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 3668, 2664, 2665, 2663}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3665, 2665, 2666, 2664, 698: 3666}, + {57: 1241, 482: 1241, 639: 3668}, + {57: 3667}, + {1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 460: 1210, 1210, 1210, 1210, 465: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 476: 1210, 1210, 479: 1210, 1210, 1210, 1210, 1210, 485: 1210, 487: 1210, 1210, 1210, 1210, 492: 1210, 494: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 530: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 570: 1210}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3669, 2665, 2666, 2664}, // 1220 - {57: 1240, 482: 1240, 639: 3669}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 3670, 2664, 2665, 2663}, - {1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 460: 1239, 1239, 1239, 1239, 465: 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 476: 1239, 1239, 479: 1239, 1239, 1239, 1239, 1239, 485: 1239, 487: 1239, 1239, 1239, 1239, 1239, 494: 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 530: 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 570: 1239, 642: 1239, 1239}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 3672}, - {57: 3673}, + {57: 1240, 482: 1240, 639: 3670}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3671, 2665, 2666, 2664}, + {1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 460: 1239, 1239, 1239, 1239, 465: 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 476: 1239, 1239, 479: 1239, 1239, 1239, 1239, 1239, 485: 1239, 487: 1239, 1239, 1239, 1239, 492: 1239, 494: 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 530: 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 570: 1239, 642: 1239, 1239}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3665, 2665, 2666, 2664, 698: 3673}, + {57: 3674}, // 1225 - {1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 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, 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}, + {1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 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, 492: 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: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3676}, + {7: 3677, 468: 3678, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {60: 3689, 105: 3685, 170: 3686, 3684, 174: 3691, 187: 3688, 486: 3696, 529: 3682, 634: 3695, 667: 3687, 3692, 3693, 672: 3694, 726: 3690, 886: 3683, 980: 3681}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 529: 3646, 648: 3378, 2665, 2666, 2664, 725: 3645, 794: 3679}, // 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}, - {57: 3731}, - {57: 278, 459: 3710, 747: 3711, 770: 3730}, - {13: 278, 57: 278, 459: 3710, 486: 278, 529: 278, 634: 278, 747: 3711, 770: 3715}, + {57: 3680}, + {1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 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, 492: 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}, + {57: 3732}, + {57: 278, 459: 3711, 747: 3712, 771: 3731}, + {13: 278, 57: 278, 459: 3711, 486: 278, 529: 278, 634: 278, 747: 3712, 771: 3716}, // 1235 {57: 1034}, {57: 1033}, - {57: 278, 459: 3710, 747: 3711, 770: 3714}, - {57: 271, 459: 3697, 747: 3698, 886: 3713, 892: 3699}, - {57: 278, 459: 3710, 747: 3711, 770: 3709}, + {57: 278, 459: 3711, 747: 3712, 771: 3715}, + {57: 271, 459: 3698, 747: 3699, 889: 3714, 894: 3700}, + {57: 278, 459: 3711, 747: 3712, 771: 3710}, // 1240 - {57: 342, 670: 3706, 3707, 1069: 3708}, - {57: 342, 670: 3706, 3707, 1069: 3705}, + {57: 342, 670: 3707, 3708, 1070: 3709}, + {57: 342, 670: 3707, 3708, 1070: 3706}, {57: 1027}, {57: 1026}, - {57: 271, 459: 3697, 747: 3698, 886: 3696, 892: 3699}, + {57: 271, 459: 3698, 747: 3699, 889: 3697, 894: 3700}, // 1245 {57: 1024}, {13: 316, 57: 316, 459: 316, 486: 316, 529: 316, 634: 316}, {13: 315, 57: 315, 459: 315, 486: 315, 529: 315, 634: 315}, {57: 1025}, - {493: 2638, 722: 2637, 730: 3700}, + {493: 2639, 722: 2638, 730: 3701}, // 1250 {270, 270, 270, 270, 270, 270, 270, 270, 270, 270, 270, 270, 270, 57: 270, 60: 270, 458: 270, 462: 270, 270, 270, 270, 467: 270, 475: 270, 478: 270, 563: 270, 571: 270, 573: 270, 629: 270, 270, 270, 633: 270, 726: 270, 728: 270}, {269, 269, 269, 269, 269, 269, 269, 269, 269, 269, 269, 269, 269, 57: 269, 60: 269, 458: 269, 462: 269, 269, 269, 269, 467: 269, 475: 269, 478: 269, 563: 269, 571: 269, 573: 269, 629: 269, 269, 269, 633: 269, 726: 269, 728: 269}, - {7: 3702, 57: 3701}, + {7: 3703, 57: 3702}, {279, 279, 279, 279, 279, 279, 279, 279, 279, 279, 279, 279, 279, 279, 57: 279, 60: 279, 101: 279, 279, 104: 279, 458: 279, 462: 279, 279, 279, 279, 467: 279, 475: 279, 478: 279, 486: 279, 515: 279, 279, 529: 279, 563: 279, 571: 279, 573: 279, 629: 279, 279, 279, 633: 279, 279, 726: 279, 728: 279}, - {493: 2638, 722: 2637, 730: 3703}, + {493: 2639, 722: 2638, 730: 3704}, // 1255 - {57: 3704}, + {57: 3705}, {268, 268, 268, 268, 268, 268, 268, 268, 268, 268, 268, 268, 268, 57: 268, 60: 268, 458: 268, 462: 268, 268, 268, 268, 467: 268, 475: 268, 478: 268, 563: 268, 571: 268, 573: 268, 629: 268, 268, 268, 633: 268, 726: 268, 728: 268}, {57: 1028}, {57: 341}, @@ -7517,781 +7518,781 @@ var ( // 1260 {57: 1029}, {57: 1030}, - {493: 2638, 722: 2637, 730: 3712}, + {493: 2639, 722: 2638, 730: 3713}, {277, 277, 277, 277, 277, 277, 277, 277, 277, 277, 277, 277, 277, 277, 57: 277, 60: 277, 101: 277, 277, 104: 277, 458: 277, 462: 277, 277, 277, 277, 467: 277, 475: 277, 478: 277, 486: 277, 515: 277, 277, 529: 277, 563: 277, 571: 277, 573: 277, 629: 277, 277, 277, 633: 277, 277, 726: 277, 728: 277}, - {57: 3701}, + {57: 3702}, // 1265 {57: 1031}, {57: 1032}, - {13: 3720, 57: 265, 486: 3721, 529: 3717, 634: 3719, 759: 3718, 784: 3716}, + {13: 3721, 57: 265, 486: 3722, 529: 3718, 634: 3720, 759: 3719, 786: 3717}, {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}, + {262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 3721, 57: 262, 458: 262, 462: 262, 262, 262, 262, 467: 262, 475: 262, 478: 262, 486: 3722, 563: 262, 571: 262, 573: 262, 629: 262, 262, 262, 633: 262, 3720, 759: 3729, 1234: 3728}, // 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, 793: 3724}, - {498: 3723}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 529: 3646, 648: 3378, 2665, 2666, 2664, 725: 3645, 794: 3725}, + {498: 3724}, {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}, + {498: 3723}, {258, 258, 258, 258, 258, 258, 258, 8: 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 58: 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 461: 258, 464: 258, 482: 258, 489: 258, 507: 258, 529: 258}, // 1275 {260, 260, 260, 260, 260, 260, 260, 8: 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 58: 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 461: 260, 464: 260, 482: 260, 489: 260, 507: 260, 529: 260}, - {267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 57: 267, 458: 267, 462: 267, 267, 267, 267, 467: 267, 475: 267, 478: 267, 529: 3725, 563: 267, 571: 267, 573: 267, 629: 267, 267, 267, 633: 267, 1233: 3726}, + {267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 57: 267, 458: 267, 462: 267, 267, 267, 267, 467: 267, 475: 267, 478: 267, 529: 3726, 563: 267, 571: 267, 573: 267, 629: 267, 267, 267, 633: 267, 1233: 3727}, {266, 266, 266, 266, 266, 266, 266, 266, 266, 266, 266, 266, 266, 57: 266, 458: 266, 462: 266, 266, 266, 266, 467: 266, 475: 266, 478: 266, 563: 266, 571: 266, 573: 266, 629: 266, 266, 266, 633: 266}, {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, 793: 3729}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 529: 3646, 648: 3378, 2665, 2666, 2664, 725: 3645, 794: 3730}, {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}, + {1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 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, 492: 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}, + {492: 3235, 494: 3233, 3234, 3232, 3230, 528: 1042, 720: 3231, 3229}, // 1285 - {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}, + {528: 3737, 1136: 3736, 1314: 3735}, + {153: 1038, 528: 3737, 531: 3743, 1136: 3742, 1180: 3741}, + {153: 1041, 528: 1041, 531: 1041}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3738}, + {492: 3235, 494: 3233, 3234, 3232, 3230, 532: 3739, 720: 3231, 3229}, // 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, 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, 3739}, - {152: 1039, 491: 3234, 494: 3232, 3233, 3231, 3229, 528: 1039, 531: 1039, 720: 3230, 3228}, - {152: 3744}, - {152: 1040, 528: 1040, 531: 1040}, - {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, 3743}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3740}, + {153: 1039, 492: 3235, 494: 3233, 3234, 3232, 3230, 528: 1039, 531: 1039, 720: 3231, 3229}, + {153: 3745}, + {153: 1040, 528: 1040, 531: 1040}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3744}, // 1295 - {152: 1037, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 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, 882: 3682, 979: 3748}, + {153: 1037, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 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, 492: 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: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3747}, + {465: 3748, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {60: 3689, 105: 3685, 170: 3686, 3684, 174: 3691, 187: 3688, 486: 3696, 529: 3682, 634: 3695, 667: 3687, 3692, 3693, 672: 3694, 726: 3690, 886: 3683, 980: 3749}, // 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}, - {1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 460: 1216, 1216, 1216, 1216, 465: 1216, 1216, 3238, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 476: 1216, 1216, 479: 1216, 1216, 1216, 1216, 1216, 485: 1216, 487: 1216, 1216, 1216, 1216, 1216, 494: 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 530: 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 570: 1216}, - {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, 3752}, - {491: 3234, 494: 3232, 3233, 3231, 3229, 508: 3753, 720: 3230, 3228}, + {57: 3750}, + {1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 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, 492: 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}, + {1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 460: 1216, 1216, 1216, 1216, 465: 1216, 1216, 3239, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 476: 1216, 1216, 479: 1216, 1216, 1216, 1216, 1216, 485: 1216, 487: 1216, 1216, 1216, 1216, 492: 1216, 494: 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 530: 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 570: 1216}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3753}, + {492: 3235, 494: 3233, 3234, 3232, 3230, 508: 3754, 720: 3231, 3229}, // 1305 - {1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 460: 1217, 1217, 1217, 1217, 465: 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 476: 1217, 1217, 479: 1217, 1217, 1217, 1217, 1217, 485: 1217, 487: 1217, 1217, 1217, 1217, 1217, 494: 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 530: 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 570: 1217}, - {1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 460: 1218, 1218, 1218, 1218, 465: 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 476: 1218, 1218, 479: 1218, 1218, 1218, 1218, 1218, 485: 1218, 487: 1218, 1218, 1218, 1218, 1218, 494: 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 530: 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 570: 1218}, - {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: 3756}, - {7: 3757}, - {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, 3758}, + {1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 460: 1217, 1217, 1217, 1217, 465: 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 476: 1217, 1217, 479: 1217, 1217, 1217, 1217, 1217, 485: 1217, 487: 1217, 1217, 1217, 1217, 492: 1217, 494: 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 530: 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 570: 1217}, + {1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 460: 1218, 1218, 1218, 1218, 465: 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 476: 1218, 1218, 479: 1218, 1218, 1218, 1218, 1218, 485: 1218, 487: 1218, 1218, 1218, 1218, 492: 1218, 494: 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 530: 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 570: 1218}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 761: 3757}, + {7: 3758}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3759}, // 1310 - {7: 1869, 57: 3759, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 460: 1219, 1219, 1219, 1219, 465: 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 476: 1219, 1219, 479: 1219, 1219, 1219, 1219, 1219, 485: 1219, 487: 1219, 1219, 1219, 1219, 1219, 494: 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 530: 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 570: 1219}, - {7: 1870, 57: 3861, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {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}, + {7: 1869, 57: 3760, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 460: 1219, 1219, 1219, 1219, 465: 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 476: 1219, 1219, 479: 1219, 1219, 1219, 1219, 1219, 485: 1219, 487: 1219, 1219, 1219, 1219, 492: 1219, 494: 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 530: 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 570: 1219}, + {7: 1870, 57: 3862, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {7: 3859}, + {7: 1222, 57: 1222, 462: 1222, 1222, 466: 780, 1222, 471: 1222, 1222, 1222, 476: 780, 780, 480: 2631, 482: 1222, 488: 2632, 490: 2628, 492: 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: 3776, 3777}, // 1315 - {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}, + {459: 3664, 561: 3781, 897: 3780, 958: 3779}, + {459: 2498, 484: 2496, 553: 2495, 632: 2491, 696: 3773, 738: 3772, 2492, 2493, 2494, 2503, 2501, 3774, 3775}, + {57: 3771, 466: 781, 476: 781, 781}, + {57: 3770}, {57: 3769}, - {57: 3768}, // 1320 - {807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 465: 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 476: 807, 807, 479: 807, 807, 807, 807, 807, 807, 807, 487: 807, 807, 807, 807, 807, 494: 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 530: 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 553: 807, 570: 807, 632: 807, 641: 807, 731: 807}, - {808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 465: 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 476: 808, 808, 479: 808, 808, 808, 808, 808, 808, 808, 487: 808, 808, 808, 808, 808, 494: 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 530: 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 553: 808, 570: 808, 632: 808, 641: 808, 731: 808}, - {809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 465: 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 476: 809, 809, 479: 809, 809, 809, 809, 809, 809, 809, 487: 809, 809, 809, 809, 809, 494: 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 530: 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 553: 809, 570: 809, 632: 809, 641: 809, 731: 809}, + {807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 465: 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 476: 807, 807, 479: 807, 807, 807, 807, 807, 807, 807, 487: 807, 807, 807, 807, 492: 807, 494: 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 530: 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 807, 553: 807, 570: 807, 632: 807, 641: 807, 731: 807}, + {808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 465: 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 476: 808, 808, 479: 808, 808, 808, 808, 808, 808, 808, 487: 808, 808, 808, 808, 492: 808, 494: 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 530: 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 808, 553: 808, 570: 808, 632: 808, 641: 808, 731: 808}, + {809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 465: 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 476: 809, 809, 479: 809, 809, 809, 809, 809, 809, 809, 487: 809, 809, 809, 809, 492: 809, 494: 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 530: 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, 553: 809, 570: 809, 632: 809, 641: 809, 731: 809}, {964, 964, 57: 964, 458: 964, 460: 964, 466: 781, 468: 964, 476: 781, 781}, - {963, 963, 57: 963, 458: 963, 460: 963, 466: 780, 468: 963, 476: 780, 780, 480: 2630, 488: 2631, 490: 2627, 753: 3775, 3776}, + {963, 963, 57: 963, 458: 963, 460: 963, 466: 780, 468: 963, 476: 780, 780, 480: 2631, 488: 2632, 490: 2628, 753: 3776, 3777}, // 1325 {793, 793, 57: 793, 458: 793, 460: 793, 468: 793}, {792, 792, 57: 792, 458: 792, 460: 792, 468: 792}, - {786, 786, 57: 786, 458: 786, 460: 786, 468: 786, 480: 2630, 488: 2631, 754: 3777}, + {786, 786, 57: 786, 458: 786, 460: 786, 468: 786, 480: 2631, 488: 2632, 754: 3778}, {785, 785, 57: 785, 458: 785, 460: 785, 468: 785}, {784, 784, 57: 784, 458: 784, 460: 784, 468: 784}, // 1330 - {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}, + {1258, 1258, 7: 3793, 57: 1258, 458: 1258, 460: 1258, 466: 1258, 468: 1258, 476: 1258, 1258, 479: 1258, 1258, 1258, 483: 1258, 488: 1258, 490: 2628, 753: 2629, 797: 3792}, {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, 790: 3786, 1304: 3785, 3784}, + {459: 3782, 826: 3783}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 1298, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3788, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3784, 784: 3787, 1304: 3786, 3785}, {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}, - {57: 3790}, - {7: 3788, 57: 1297}, + {1294, 1294, 7: 1294, 57: 1294, 458: 1294, 468: 1294, 480: 1294, 489: 1294, 1294, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {57: 3791}, + {7: 3789, 57: 1297}, {7: 1295, 57: 1295}, - {1293, 1293, 7: 1293, 57: 1293, 458: 1293, 3671, 468: 1293, 480: 1293, 489: 1293, 1293}, + {1293, 1293, 7: 1293, 57: 1293, 458: 1293, 3672, 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, 790: 3789}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3788, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3784, 784: 3790}, {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, 896: 3793}, + {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, 492: 1299}, + {834, 834, 57: 834, 458: 834, 460: 834, 466: 834, 468: 834, 476: 834, 834, 479: 834, 2631, 834, 483: 834, 488: 2632, 754: 2633, 815: 3795}, + {561: 3781, 897: 3794}, // 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, 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}, + {806, 806, 57: 806, 458: 806, 460: 806, 466: 806, 468: 806, 476: 806, 806, 479: 3797, 481: 806, 483: 3798, 871: 3796}, + {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3823, 872: 3822}, + {271: 3803, 641: 3802}, + {530: 3799}, // 1350 - {271: 3799}, - {201: 3800}, + {271: 3800}, + {201: 3801}, {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, 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}, + {797, 797, 57: 797, 137: 797, 148: 797, 168: 797, 458: 797, 460: 797, 466: 797, 468: 797, 476: 797, 797, 481: 797, 1063: 3805, 3816}, + {797, 797, 57: 797, 137: 797, 148: 797, 458: 797, 460: 797, 466: 797, 468: 797, 476: 797, 797, 481: 797, 1063: 3805, 3804}, // 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, 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}, + {804, 804, 57: 804, 137: 3814, 148: 3813, 458: 804, 460: 804, 466: 804, 468: 804, 476: 804, 804, 481: 804}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 3807, 782: 3808}, + {1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 1018, 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, 3811, 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, 168: 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: 3809, 57: 796, 137: 796, 148: 796, 168: 796, 458: 796, 460: 796, 466: 796, 468: 796, 476: 796, 796, 481: 796}, // 1360 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 3809}, - {1015, 1015, 7: 1015, 57: 1015, 137: 1015, 148: 1015, 155: 1015, 167: 1015, 458: 1015, 460: 1015, 466: 1015, 468: 1015, 476: 1015, 1015, 481: 1015, 487: 1015, 637: 1015, 1015, 657: 1015, 659: 1015}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 3811, 2664, 2665, 2663}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 3810}, + {1015, 1015, 7: 1015, 57: 1015, 137: 1015, 148: 1015, 156: 1015, 168: 1015, 458: 1015, 460: 1015, 466: 1015, 468: 1015, 476: 1015, 1015, 481: 1015, 487: 1015, 637: 1015, 1015, 657: 1015, 659: 1015}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3812, 2665, 2666, 2664}, {1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 530: 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 553: 1017, 563: 1017, 570: 1017, 1017, 1017, 1017, 629: 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 1017, 640: 1017, 1017, 644: 1017, 1017, 1017, 653: 1017, 656: 1017, 1017, 1017, 1017, 1017, 1017, 666: 1017, 673: 1017, 1017, 676: 1017, 691: 1017}, {801, 801, 57: 801, 458: 801, 460: 801, 466: 801, 468: 801, 476: 801, 801, 481: 801}, // 1365 - {252: 3814}, + {253: 3815}, {799, 799, 57: 799, 458: 799, 460: 799, 466: 799, 468: 799, 476: 799, 799, 481: 799}, - {805, 805, 57: 805, 137: 3818, 148: 3816, 167: 3817, 458: 805, 460: 805, 466: 805, 468: 805, 476: 805, 805, 481: 805}, + {805, 805, 57: 805, 137: 3819, 148: 3817, 168: 3818, 458: 805, 460: 805, 466: 805, 468: 805, 476: 805, 805, 481: 805}, {803, 803, 57: 803, 458: 803, 460: 803, 466: 803, 468: 803, 476: 803, 803, 481: 803}, - {493: 2638, 722: 3820}, + {493: 2639, 722: 3821}, // 1370 - {252: 3819}, + {253: 3820}, {800, 800, 57: 800, 458: 800, 460: 800, 466: 800, 468: 800, 476: 800, 800, 481: 800}, {802, 802, 57: 802, 458: 802, 460: 802, 466: 802, 468: 802, 476: 802, 802, 481: 802}, {965, 965, 57: 965, 458: 965, 460: 965, 466: 965, 468: 965, 476: 965, 965}, - {1245: 3823}, + {1245: 3824}, // 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, 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}, + {461: 3825}, + {94, 94, 57: 94, 98: 3829, 107: 3828, 458: 94, 460: 94, 466: 94, 468: 94, 476: 94, 94, 645: 94, 820: 3827, 1029: 3826}, + {81, 81, 57: 81, 458: 81, 460: 81, 466: 81, 468: 81, 476: 81, 81, 645: 3850, 928: 3849}, + {767: 3832, 775: 3834, 780: 3835, 3833, 1028: 3831, 1187: 3830}, + {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, 767: 92, 775: 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, 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}, - {772: 3842}, + {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, 767: 91, 775: 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, 767: 3832, 775: 3834, 780: 3835, 3833, 1028: 3848}, + {89, 89, 57: 89, 458: 89, 89, 89, 466: 89, 468: 89, 474: 89, 476: 89, 89, 498: 89, 645: 89, 767: 89, 775: 89, 780: 89, 89}, + {646: 3846}, + {775: 3843}, // 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, 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}, + {646: 3841}, + {646: 3836}, + {461: 3838, 562: 3839, 566: 3840, 837: 3837}, + {85, 85, 57: 85, 458: 85, 85, 85, 466: 85, 468: 85, 474: 85, 476: 85, 85, 498: 85, 645: 85, 767: 85, 775: 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, 767: 84, 775: 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, 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, 772: 86, 780: 86, 86}, - {646: 3843}, + {83, 83, 57: 83, 458: 83, 83, 83, 466: 83, 468: 83, 474: 83, 476: 83, 83, 498: 83, 645: 83, 767: 83, 775: 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, 767: 82, 775: 82, 780: 82, 82}, + {461: 3838, 562: 3839, 566: 3840, 837: 3842}, + {86, 86, 57: 86, 458: 86, 86, 86, 466: 86, 468: 86, 474: 86, 476: 86, 86, 498: 86, 645: 86, 767: 86, 775: 86, 780: 86, 86}, + {646: 3844}, // 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, 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, 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}, + {461: 3838, 562: 3839, 566: 3840, 837: 3845}, + {87, 87, 57: 87, 458: 87, 87, 87, 466: 87, 468: 87, 474: 87, 476: 87, 87, 498: 87, 645: 87, 767: 87, 775: 87, 780: 87, 87}, + {461: 3838, 562: 3839, 566: 3840, 837: 3847}, + {88, 88, 57: 88, 458: 88, 88, 88, 466: 88, 468: 88, 474: 88, 476: 88, 88, 498: 88, 645: 88, 767: 88, 775: 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, 767: 90, 775: 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}, - {77, 77, 57: 77, 458: 77, 77, 77, 466: 77, 468: 77, 474: 77, 476: 77, 77, 498: 77, 765: 3855, 1218: 3854}, - {646: 3852}, - {461: 3837, 562: 3838, 566: 3839, 836: 3853}, + {79, 79, 57: 79, 458: 79, 79, 79, 466: 79, 468: 79, 474: 79, 476: 79, 79, 498: 79, 767: 79, 1278: 3851, 3852}, + {77, 77, 57: 77, 458: 77, 77, 77, 466: 77, 468: 77, 474: 77, 476: 77, 77, 498: 77, 767: 3856, 1218: 3855}, + {646: 3853}, + {461: 3838, 562: 3839, 566: 3840, 837: 3854}, // 1405 - {78, 78, 57: 78, 458: 78, 78, 78, 466: 78, 468: 78, 474: 78, 476: 78, 78, 498: 78, 765: 78}, + {78, 78, 57: 78, 458: 78, 78, 78, 466: 78, 468: 78, 474: 78, 476: 78, 78, 498: 78, 767: 78}, {80, 80, 57: 80, 458: 80, 80, 80, 466: 80, 468: 80, 474: 80, 476: 80, 80, 498: 80}, - {646: 3856}, - {461: 3837, 562: 3838, 566: 3839, 836: 3857}, + {646: 3857}, + {461: 3838, 562: 3839, 566: 3840, 837: 3858}, {76, 76, 57: 76, 458: 76, 76, 76, 466: 76, 468: 76, 474: 76, 476: 76, 76, 498: 76}, // 1410 - {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, 3859}, - {7: 1869, 57: 3860, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 460: 1220, 1220, 1220, 1220, 465: 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 476: 1220, 1220, 479: 1220, 1220, 1220, 1220, 1220, 485: 1220, 487: 1220, 1220, 1220, 1220, 1220, 494: 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 530: 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 570: 1220}, - {1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 460: 1221, 1221, 1221, 1221, 465: 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 476: 1221, 1221, 479: 1221, 1221, 1221, 1221, 1221, 485: 1221, 487: 1221, 1221, 1221, 1221, 1221, 494: 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 530: 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 570: 1221}, - {1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 460: 1223, 1223, 1223, 1223, 465: 1223, 1223, 3238, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 476: 1223, 1223, 479: 1223, 1223, 1223, 1223, 1223, 485: 1223, 487: 1223, 1223, 1223, 1223, 1223, 494: 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 530: 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 570: 1223}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3860}, + {7: 1869, 57: 3861, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 460: 1220, 1220, 1220, 1220, 465: 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 476: 1220, 1220, 479: 1220, 1220, 1220, 1220, 1220, 485: 1220, 487: 1220, 1220, 1220, 1220, 492: 1220, 494: 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 530: 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 570: 1220}, + {1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 460: 1221, 1221, 1221, 1221, 465: 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 476: 1221, 1221, 479: 1221, 1221, 1221, 1221, 1221, 485: 1221, 487: 1221, 1221, 1221, 1221, 492: 1221, 494: 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 530: 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 570: 1221}, + {1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 460: 1223, 1223, 1223, 1223, 465: 1223, 1223, 3239, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 476: 1223, 1223, 479: 1223, 1223, 1223, 1223, 1223, 485: 1223, 487: 1223, 1223, 1223, 1223, 492: 1223, 494: 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 530: 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 570: 1223}, // 1415 - {1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 460: 1225, 1225, 1225, 1225, 465: 1225, 1225, 3238, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 476: 1225, 1225, 479: 1225, 1225, 1225, 1225, 1225, 485: 1225, 487: 1225, 1225, 1225, 1225, 1225, 494: 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 530: 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 570: 1225}, - {1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 460: 1226, 1226, 1226, 1226, 465: 1226, 1226, 3238, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 476: 1226, 1226, 479: 1226, 1226, 1226, 1226, 1226, 485: 1226, 487: 1226, 1226, 1226, 1226, 1226, 494: 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 530: 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 570: 1226}, - {1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 460: 1227, 1227, 1227, 1227, 465: 1227, 1227, 3238, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 476: 1227, 1227, 479: 1227, 1227, 1227, 1227, 1227, 485: 1227, 487: 1227, 1227, 1227, 1227, 1227, 494: 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 530: 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 570: 1227}, - {1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 460: 1228, 1228, 1228, 1228, 465: 1228, 1228, 3238, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 476: 1228, 1228, 479: 1228, 1228, 1228, 1228, 1228, 485: 1228, 487: 1228, 1228, 1228, 1228, 1228, 494: 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 530: 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 570: 1228}, - {461: 3870}, + {1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 460: 1225, 1225, 1225, 1225, 465: 1225, 1225, 3239, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 476: 1225, 1225, 479: 1225, 1225, 1225, 1225, 1225, 485: 1225, 487: 1225, 1225, 1225, 1225, 492: 1225, 494: 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 530: 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 570: 1225}, + {1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 460: 1226, 1226, 1226, 1226, 465: 1226, 1226, 3239, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 476: 1226, 1226, 479: 1226, 1226, 1226, 1226, 1226, 485: 1226, 487: 1226, 1226, 1226, 1226, 492: 1226, 494: 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 530: 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 570: 1226}, + {1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 460: 1227, 1227, 1227, 1227, 465: 1227, 1227, 3239, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 476: 1227, 1227, 479: 1227, 1227, 1227, 1227, 1227, 485: 1227, 487: 1227, 1227, 1227, 1227, 492: 1227, 494: 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 530: 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 570: 1227}, + {1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 460: 1228, 1228, 1228, 1228, 465: 1228, 1228, 3239, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 476: 1228, 1228, 479: 1228, 1228, 1228, 1228, 1228, 485: 1228, 487: 1228, 1228, 1228, 1228, 492: 1228, 494: 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 530: 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 570: 1228}, + {461: 3871}, // 1420 - {461: 3869}, - {1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 460: 1208, 1208, 1208, 1208, 465: 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 476: 1208, 1208, 479: 1208, 1208, 1208, 1208, 1208, 485: 1208, 487: 1208, 1208, 1208, 1208, 1208, 494: 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 530: 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 570: 1208}, - {1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 460: 1209, 1209, 1209, 1209, 465: 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 476: 1209, 1209, 479: 1209, 1209, 1209, 1209, 1209, 485: 1209, 487: 1209, 1209, 1209, 1209, 1209, 494: 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 530: 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 570: 1209}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 3872, 2664, 2665, 2663}, - {1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 3873, 1240, 1240, 1240, 1240, 465: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 476: 1240, 1240, 479: 1240, 1240, 1240, 1240, 1240, 485: 1240, 487: 1240, 1240, 1240, 1240, 1240, 494: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 530: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 570: 1240, 639: 3669, 642: 1240, 1240}, + {461: 3870}, + {1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 460: 1208, 1208, 1208, 1208, 465: 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 476: 1208, 1208, 479: 1208, 1208, 1208, 1208, 1208, 485: 1208, 487: 1208, 1208, 1208, 1208, 492: 1208, 494: 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 530: 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 1208, 570: 1208}, + {1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 460: 1209, 1209, 1209, 1209, 465: 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 476: 1209, 1209, 479: 1209, 1209, 1209, 1209, 1209, 485: 1209, 487: 1209, 1209, 1209, 1209, 492: 1209, 494: 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 530: 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 570: 1209}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3873, 2665, 2666, 2664}, + {1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 3874, 1240, 1240, 1240, 1240, 465: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 476: 1240, 1240, 479: 1240, 1240, 1240, 1240, 1240, 485: 1240, 487: 1240, 1240, 1240, 1240, 492: 1240, 494: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 530: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 570: 1240, 639: 3670, 642: 1240, 1240}, // 1425 - {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, 1866, 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: 3636, 811: 3874}, - {57: 3875}, - {1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 460: 1076, 1076, 1076, 1076, 465: 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 476: 1076, 1076, 479: 1076, 1076, 1076, 1076, 1076, 485: 1076, 487: 1076, 1076, 1076, 1076, 1076, 494: 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 530: 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 570: 1076}, - {1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 460: 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 485: 1278, 487: 1278, 1278, 1278, 1278, 1278, 494: 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 530: 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 563: 1278, 570: 1278, 1278, 573: 1278, 629: 1278, 1278, 1278, 633: 1278}, - {1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 460: 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 485: 1275, 487: 1275, 1275, 1275, 1275, 1275, 494: 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 530: 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 563: 1275, 570: 1275, 1275, 573: 1275, 629: 1275, 1275, 1275, 633: 1275}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 1866, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 761: 3637, 812: 3875}, + {57: 3876}, + {1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 460: 1076, 1076, 1076, 1076, 465: 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 476: 1076, 1076, 479: 1076, 1076, 1076, 1076, 1076, 485: 1076, 487: 1076, 1076, 1076, 1076, 492: 1076, 494: 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 530: 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 570: 1076}, + {1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 460: 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 485: 1278, 487: 1278, 1278, 1278, 1278, 492: 1278, 494: 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 530: 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 1278, 563: 1278, 570: 1278, 1278, 573: 1278, 629: 1278, 1278, 1278, 633: 1278}, + {1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 460: 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 485: 1275, 487: 1275, 1275, 1275, 1275, 492: 1275, 494: 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 530: 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 1275, 563: 1275, 570: 1275, 1275, 573: 1275, 629: 1275, 1275, 1275, 633: 1275}, // 1430 - {1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 460: 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 485: 1274, 487: 1274, 1274, 1274, 1274, 1274, 494: 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 530: 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 563: 1274, 570: 1274, 1274, 573: 1274, 629: 1274, 1274, 1274, 633: 1274}, - {1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 460: 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 485: 1272, 487: 1272, 1272, 1272, 1272, 1272, 494: 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 530: 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 563: 1272, 570: 1272, 1272, 573: 1272, 629: 1272, 1272, 1272, 633: 1272}, - {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, 638: 3882, 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, 3881}, - {57: 3886, 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, 3883}, + {1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 460: 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 485: 1274, 487: 1274, 1274, 1274, 1274, 492: 1274, 494: 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 530: 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 1274, 563: 1274, 570: 1274, 1274, 573: 1274, 629: 1274, 1274, 1274, 633: 1274}, + {1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 460: 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 485: 1272, 487: 1272, 1272, 1272, 1272, 492: 1272, 494: 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 530: 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 1272, 563: 1272, 570: 1272, 1272, 573: 1272, 629: 1272, 1272, 1272, 633: 1272}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 638: 3883, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3882}, + {57: 3887, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3884}, // 1435 - {57: 3884, 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: 3885}, - {1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 460: 1084, 1084, 1084, 1084, 465: 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 476: 1084, 1084, 479: 1084, 1084, 1084, 1084, 1084, 485: 1084, 487: 1084, 1084, 1084, 1084, 1084, 494: 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 530: 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 570: 1084}, - {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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: 3887}, - {1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 460: 1085, 1085, 1085, 1085, 465: 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 476: 1085, 1085, 479: 1085, 1085, 1085, 1085, 1085, 485: 1085, 487: 1085, 1085, 1085, 1085, 1085, 494: 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 530: 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 570: 1085}, + {57: 3885, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3886}, + {1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 460: 1084, 1084, 1084, 1084, 465: 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 476: 1084, 1084, 479: 1084, 1084, 1084, 1084, 1084, 485: 1084, 487: 1084, 1084, 1084, 1084, 492: 1084, 494: 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 530: 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 570: 1084}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3888}, + {1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 460: 1085, 1085, 1085, 1085, 465: 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 476: 1085, 1085, 479: 1085, 1085, 1085, 1085, 1085, 485: 1085, 487: 1085, 1085, 1085, 1085, 492: 1085, 494: 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 530: 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 1085, 570: 1085}, // 1440 - {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, 638: 3890, 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, 3889}, - {7: 3900, 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, 3891}, - {7: 3892, 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, 638: 3894, 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, 3893}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 638: 3891, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3890}, + {7: 3901, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3892}, + {7: 3893, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 638: 3895, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3894}, // 1445 - {57: 3898, 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, 3895}, - {57: 3896, 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: 3897}, - {1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 460: 1080, 1080, 1080, 1080, 465: 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 476: 1080, 1080, 479: 1080, 1080, 1080, 1080, 1080, 485: 1080, 487: 1080, 1080, 1080, 1080, 1080, 494: 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 530: 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 570: 1080}, + {57: 3899, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3896}, + {57: 3897, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3898}, + {1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 460: 1080, 1080, 1080, 1080, 465: 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 476: 1080, 1080, 479: 1080, 1080, 1080, 1080, 1080, 485: 1080, 487: 1080, 1080, 1080, 1080, 492: 1080, 494: 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 530: 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 570: 1080}, // 1450 - {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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: 3899}, - {1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 460: 1082, 1082, 1082, 1082, 465: 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 476: 1082, 1082, 479: 1082, 1082, 1082, 1082, 1082, 485: 1082, 487: 1082, 1082, 1082, 1082, 1082, 494: 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 530: 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 570: 1082}, - {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, 638: 3902, 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, 3901}, - {57: 3906, 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, 3903}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3900}, + {1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 460: 1082, 1082, 1082, 1082, 465: 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 476: 1082, 1082, 479: 1082, 1082, 1082, 1082, 1082, 485: 1082, 487: 1082, 1082, 1082, 1082, 492: 1082, 494: 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 530: 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 570: 1082}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 638: 3903, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3902}, + {57: 3907, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3904}, // 1455 - {57: 3904, 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: 3905}, - {1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 460: 1081, 1081, 1081, 1081, 465: 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 476: 1081, 1081, 479: 1081, 1081, 1081, 1081, 1081, 485: 1081, 487: 1081, 1081, 1081, 1081, 1081, 494: 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 530: 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 570: 1081}, - {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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}, + {57: 3905, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3906}, + {1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 460: 1081, 1081, 1081, 1081, 465: 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 476: 1081, 1081, 479: 1081, 1081, 1081, 1081, 1081, 485: 1081, 487: 1081, 1081, 1081, 1081, 492: 1081, 494: 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 530: 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 570: 1081}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3908}, + {1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 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, 492: 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, 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}, - {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, 3913}, + {105: 3351, 3347, 108: 3344, 3359, 111: 3346, 3343, 3345, 3349, 3350, 3355, 3354, 3353, 3357, 3358, 3352, 3356, 3348, 793: 3910}, + {7: 3911}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3912}, + {7: 3913, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3914}, // 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, 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}, + {57: 3915, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 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, 492: 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: 3351, 3347, 108: 3344, 3359, 111: 3346, 3343, 3345, 3349, 3350, 3355, 3354, 3353, 3357, 3358, 3352, 3356, 3348, 793: 3917}, + {7: 3918}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3919}, // 1470 - {7: 3919, 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, 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, 1197: 3923}, + {7: 3920, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3921}, + {57: 3922, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 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, 492: 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: 3926, 3925, 187: 3927, 213: 3928, 1197: 3924}, // 1475 - {7: 3928}, + {7: 3929}, {7: 1120}, {7: 1119}, {7: 1118}, {7: 1117}, // 1480 - {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, 3929}, - {57: 3930, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 460: 1137, 1137, 1137, 1137, 465: 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 476: 1137, 1137, 479: 1137, 1137, 1137, 1137, 1137, 485: 1137, 487: 1137, 1137, 1137, 1137, 1137, 494: 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 530: 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 570: 1137}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 3932}, - {7: 3933}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3930}, + {57: 3931, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 460: 1137, 1137, 1137, 1137, 465: 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 476: 1137, 1137, 479: 1137, 1137, 1137, 1137, 1137, 485: 1137, 487: 1137, 1137, 1137, 1137, 492: 1137, 494: 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 530: 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 570: 1137}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 3933}, + {7: 3934}, // 1485 - {471: 3938, 3937, 493: 2638, 722: 3934, 748: 3936, 799: 3935}, + {471: 3939, 3938, 493: 2639, 722: 3935, 748: 3937, 799: 3936}, {1921, 1921, 4: 1921, 1921, 1921, 1921, 13: 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 81: 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 1921, 106: 1921, 126: 1921, 1921, 1921, 1921, 464: 1921, 466: 1921, 1921, 480: 1921, 485: 1921, 1921, 488: 1921, 1921, 634: 1921, 1921, 644: 1921}, - {57: 3941}, + {57: 3942}, {29, 29, 4: 29, 29, 29, 13: 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 81: 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 464: 29, 466: 29, 29, 485: 29, 29, 634: 29, 29, 644: 29}, - {493: 2638, 722: 3934, 748: 3940}, + {493: 2639, 722: 3935, 748: 3941}, // 1490 - {493: 2638, 722: 3939}, + {493: 2639, 722: 3940}, {27, 27, 4: 27, 27, 27, 13: 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 81: 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 464: 27, 466: 27, 27, 485: 27, 27, 634: 27, 27, 644: 27}, {28, 28, 4: 28, 28, 28, 13: 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 81: 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 464: 28, 466: 28, 28, 485: 28, 28, 634: 28, 28, 644: 28}, - {1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 460: 1108, 1108, 1108, 1108, 465: 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 476: 1108, 1108, 479: 1108, 1108, 1108, 1108, 1108, 485: 1108, 487: 1108, 1108, 1108, 1108, 1108, 494: 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 530: 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 570: 1108}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 3943}, + {1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 460: 1108, 1108, 1108, 1108, 465: 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 476: 1108, 1108, 479: 1108, 1108, 1108, 1108, 1108, 485: 1108, 487: 1108, 1108, 1108, 1108, 492: 1108, 494: 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 530: 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 570: 1108}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 3944}, // 1495 - {57: 3944}, - {1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 460: 1109, 1109, 1109, 1109, 465: 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 476: 1109, 1109, 479: 1109, 1109, 1109, 1109, 1109, 485: 1109, 487: 1109, 1109, 1109, 1109, 1109, 494: 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 530: 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 570: 1109}, - {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, 3946}, - {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}, + {57: 3945}, + {1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 460: 1109, 1109, 1109, 1109, 465: 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 476: 1109, 1109, 479: 1109, 1109, 1109, 1109, 1109, 485: 1109, 487: 1109, 1109, 1109, 1109, 492: 1109, 494: 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 530: 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 570: 1109}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3947}, + {57: 3948, 465: 3949, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 1125, 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, 492: 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, 882: 3949}, - {459: 3710, 747: 3953}, - {459: 3710, 747: 3951}, - {57: 3952}, - {1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 460: 1123, 1123, 1123, 1123, 465: 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 476: 1123, 1123, 479: 1123, 1123, 1123, 1123, 1123, 485: 1123, 487: 1123, 1123, 1123, 1123, 1123, 494: 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 530: 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 570: 1123}, + {486: 3696, 529: 3951, 634: 3695, 886: 3950}, + {459: 3711, 747: 3954}, + {459: 3711, 747: 3952}, + {57: 3953}, + {1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 460: 1123, 1123, 1123, 1123, 465: 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 476: 1123, 1123, 479: 1123, 1123, 1123, 1123, 1123, 485: 1123, 487: 1123, 1123, 1123, 1123, 492: 1123, 494: 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 530: 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 1123, 570: 1123}, // 1505 - {57: 3954}, - {1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 460: 1124, 1124, 1124, 1124, 465: 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 476: 1124, 1124, 479: 1124, 1124, 1124, 1124, 1124, 485: 1124, 487: 1124, 1124, 1124, 1124, 1124, 494: 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 530: 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 570: 1124}, - {1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 460: 1147, 1147, 1147, 1147, 465: 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 476: 1147, 1147, 479: 1147, 1147, 1147, 1147, 1147, 485: 1147, 487: 1147, 1147, 1147, 1147, 1147, 494: 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 530: 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 570: 1147}, - {1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 460: 1148, 1148, 1148, 1148, 465: 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 476: 1148, 1148, 479: 1148, 1148, 1148, 1148, 1148, 485: 1148, 487: 1148, 1148, 1148, 1148, 1148, 494: 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 530: 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 570: 1148}, - {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, 1866, 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: 3636, 811: 3958}, + {57: 3955}, + {1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 460: 1124, 1124, 1124, 1124, 465: 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 476: 1124, 1124, 479: 1124, 1124, 1124, 1124, 1124, 485: 1124, 487: 1124, 1124, 1124, 1124, 492: 1124, 494: 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 530: 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 1124, 570: 1124}, + {1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 460: 1147, 1147, 1147, 1147, 465: 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 476: 1147, 1147, 479: 1147, 1147, 1147, 1147, 1147, 485: 1147, 487: 1147, 1147, 1147, 1147, 492: 1147, 494: 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 530: 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 570: 1147}, + {1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 460: 1148, 1148, 1148, 1148, 465: 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 476: 1148, 1148, 479: 1148, 1148, 1148, 1148, 1148, 485: 1148, 487: 1148, 1148, 1148, 1148, 492: 1148, 494: 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 530: 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 570: 1148}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 1866, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 761: 3637, 812: 3959}, // 1510 - {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, 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}, + {57: 3960}, + {1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 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, 492: 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, 492: 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, 491: 1203, 493: 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: 3433, 732: 3431, 3432, 768: 3434, 774: 3435, 800: 3963, 802: 3436}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3964}, // 1515 - {57: 3964, 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: 3965}, - {1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 460: 1106, 1106, 1106, 1106, 465: 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 476: 1106, 1106, 479: 1106, 1106, 1106, 1106, 1106, 485: 1106, 487: 1106, 1106, 1106, 1106, 1106, 494: 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 530: 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 570: 1106}, - {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, 1866, 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: 3636, 811: 3967}, - {57: 3968}, + {57: 3965, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 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, 492: 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: 3251, 734: 3441, 752: 3966}, + {1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 460: 1106, 1106, 1106, 1106, 465: 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 476: 1106, 1106, 479: 1106, 1106, 1106, 1106, 1106, 485: 1106, 487: 1106, 1106, 1106, 1106, 492: 1106, 494: 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 530: 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 570: 1106}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 1866, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 761: 3637, 812: 3968}, + {57: 3969}, // 1520 - {1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 460: 1077, 1077, 1077, 1077, 465: 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 476: 1077, 1077, 479: 1077, 1077, 1077, 1077, 1077, 485: 1077, 487: 1077, 1077, 1077, 1077, 1077, 494: 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 530: 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 570: 1077}, - {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, 1089: 3997}, + {1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 460: 1077, 1077, 1077, 1077, 465: 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 476: 1077, 1077, 479: 1077, 1077, 1077, 1077, 1077, 485: 1077, 487: 1077, 1077, 1077, 1077, 492: 1077, 494: 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 530: 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 570: 1077}, + {146: 2233, 167: 2233, 183: 2233, 478: 2233, 507: 2233, 530: 2233, 541: 2233, 550: 2233, 2233, 557: 2233, 2233, 569: 2233}, + {146: 2232, 167: 2232, 183: 2232, 478: 2232, 507: 2232, 530: 2232, 541: 2232, 550: 2232, 2232, 557: 2232, 2232, 569: 2232}, + {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, 491: 1848, 493: 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: 3997, 530: 3996, 541: 3995, 550: 3981, 3982, 1090: 3998}, // 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}, - {2: 1840, 1840, 1840, 1840, 1840, 8: 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 58: 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 459: 1840, 461: 1840, 1840, 464: 1840, 469: 1840, 1840, 1840, 1840, 1840, 478: 1840, 484: 1840, 486: 1840, 492: 1840, 1840, 529: 1840, 552: 1840, 554: 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 564: 1840, 1840, 1840, 1840, 1840, 1840, 572: 1840, 574: 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840}, - {459: 3990, 696: 3991}, - {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, 3987}, + {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, 491: 1842, 493: 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}, + {2: 1840, 1840, 1840, 1840, 1840, 8: 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 58: 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 459: 1840, 461: 1840, 1840, 464: 1840, 469: 1840, 1840, 1840, 1840, 1840, 478: 1840, 484: 1840, 486: 1840, 491: 1840, 493: 1840, 529: 1840, 552: 1840, 554: 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 564: 1840, 1840, 1840, 1840, 1840, 1840, 572: 1840, 574: 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840}, + {459: 3991, 696: 3992}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 3988}, // 1530 - {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, 3983, 3140, 3223, 3139, 3136}, - {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, 3982, 3140, 3223, 3139, 3136}, - {2: 1829, 1829, 1829, 1829, 1829, 8: 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 58: 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 459: 1829, 461: 1829, 1829, 464: 1829, 469: 1829, 1829, 1829, 1829, 1829, 478: 1829, 484: 1829, 486: 1829, 492: 1829, 1829, 529: 1829, 552: 1829, 554: 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 564: 1829, 1829, 1829, 1829, 1829, 1829, 572: 1829, 574: 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829}, - {2: 1828, 1828, 1828, 1828, 1828, 8: 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 58: 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 459: 1828, 461: 1828, 1828, 464: 1828, 469: 1828, 1828, 1828, 1828, 1828, 478: 1828, 484: 1828, 486: 1828, 492: 1828, 1828, 529: 1828, 552: 1828, 554: 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 564: 1828, 1828, 1828, 1828, 1828, 1828, 572: 1828, 574: 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828}, - {1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 460: 1831, 1831, 465: 1831, 1831, 3238, 1831, 1831, 1831, 474: 1831, 476: 1831, 1831, 479: 1831, 1831, 1831, 1831, 1831, 485: 1831, 487: 1831, 1831, 1831, 1831, 1831, 494: 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 508: 1831, 510: 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 531: 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 570: 3239}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3984, 3141, 3224, 3140, 3137}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3983, 3141, 3224, 3140, 3137}, + {2: 1829, 1829, 1829, 1829, 1829, 8: 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 58: 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 459: 1829, 461: 1829, 1829, 464: 1829, 469: 1829, 1829, 1829, 1829, 1829, 478: 1829, 484: 1829, 486: 1829, 491: 1829, 493: 1829, 529: 1829, 552: 1829, 554: 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 564: 1829, 1829, 1829, 1829, 1829, 1829, 572: 1829, 574: 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829, 1829}, + {2: 1828, 1828, 1828, 1828, 1828, 8: 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 58: 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 459: 1828, 461: 1828, 1828, 464: 1828, 469: 1828, 1828, 1828, 1828, 1828, 478: 1828, 484: 1828, 486: 1828, 491: 1828, 493: 1828, 529: 1828, 552: 1828, 554: 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 564: 1828, 1828, 1828, 1828, 1828, 1828, 572: 1828, 574: 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828, 1828}, + {1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 460: 1831, 1831, 465: 1831, 1831, 3239, 1831, 1831, 1831, 474: 1831, 476: 1831, 1831, 479: 1831, 1831, 1831, 1831, 1831, 485: 1831, 487: 1831, 1831, 1831, 1831, 492: 1831, 494: 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 508: 1831, 510: 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 531: 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 1831, 570: 3240}, // 1535 - {1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 3985, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 460: 1827, 1827, 465: 1827, 1827, 3238, 1827, 1827, 1827, 474: 1827, 476: 1827, 1827, 479: 1827, 1827, 1827, 1827, 1827, 485: 1827, 487: 1827, 1827, 1827, 1827, 1827, 494: 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 508: 1827, 510: 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 531: 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 570: 3239, 1215: 3984}, - {1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 460: 1832, 1832, 465: 1832, 1832, 468: 1832, 1832, 1832, 474: 1832, 476: 1832, 1832, 479: 1832, 1832, 1832, 1832, 1832, 485: 1832, 487: 1832, 1832, 1832, 1832, 1832, 494: 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 508: 1832, 510: 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 531: 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832}, - {461: 3986}, - {1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 460: 1826, 1826, 465: 1826, 1826, 468: 1826, 1826, 1826, 474: 1826, 476: 1826, 1826, 479: 1826, 1826, 1826, 1826, 1826, 485: 1826, 487: 1826, 1826, 1826, 1826, 1826, 494: 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 508: 1826, 510: 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 531: 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826}, - {471: 3573, 3572, 3578, 491: 3988, 509: 3574, 542: 3575, 3576, 3569, 3579, 3568, 3577, 3570, 3571}, + {1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 3986, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 460: 1827, 1827, 465: 1827, 1827, 3239, 1827, 1827, 1827, 474: 1827, 476: 1827, 1827, 479: 1827, 1827, 1827, 1827, 1827, 485: 1827, 487: 1827, 1827, 1827, 1827, 492: 1827, 494: 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 508: 1827, 510: 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 531: 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 1827, 570: 3240, 1215: 3985}, + {1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 460: 1832, 1832, 465: 1832, 1832, 468: 1832, 1832, 1832, 474: 1832, 476: 1832, 1832, 479: 1832, 1832, 1832, 1832, 1832, 485: 1832, 487: 1832, 1832, 1832, 1832, 492: 1832, 494: 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 508: 1832, 510: 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 531: 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832, 1832}, + {461: 3987}, + {1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 460: 1826, 1826, 465: 1826, 1826, 468: 1826, 1826, 1826, 474: 1826, 476: 1826, 1826, 479: 1826, 1826, 1826, 1826, 1826, 485: 1826, 487: 1826, 1826, 1826, 1826, 492: 1826, 494: 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 508: 1826, 510: 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 531: 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826, 1826}, + {471: 3574, 3573, 3579, 492: 3989, 509: 3575, 542: 3576, 3577, 3570, 3580, 3569, 3578, 3571, 3572}, // 1540 - {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, 2661, 3989}, - {1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 460: 1833, 1833, 465: 1833, 1833, 468: 1833, 1833, 1833, 474: 1833, 476: 1833, 1833, 479: 1833, 1833, 1833, 1833, 1833, 485: 1833, 487: 1833, 1833, 1833, 1833, 1833, 494: 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 508: 1833, 510: 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 531: 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833}, - {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, 2496, 3127, 3145, 2657, 3155, 469: 3160, 3164, 3143, 3144, 3182, 478: 3118, 484: 3763, 486: 3180, 492: 3163, 3122, 529: 3151, 552: 3158, 2494, 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, 632: 2490, 636: 2658, 648: 3131, 2664, 2665, 2663, 696: 3762, 3224, 3132, 3137, 3123, 3197, 3135, 3133, 3134, 3175, 3186, 3185, 3179, 3177, 3191, 3130, 3140, 3223, 3139, 3136, 2661, 2660, 2659, 3474, 738: 3765, 2491, 2492, 2493, 2502, 744: 2500, 2499, 2498, 750: 3767, 3766, 3764, 761: 3992}, - {1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 460: 1834, 1834, 465: 1834, 1834, 468: 1834, 1834, 1834, 474: 1834, 476: 1834, 1834, 479: 1834, 1834, 1834, 1834, 1834, 485: 1834, 487: 1834, 1834, 1834, 1834, 1834, 494: 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 508: 1834, 510: 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 531: 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834}, - {7: 3476, 57: 3993}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 3990}, + {1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 460: 1833, 1833, 465: 1833, 1833, 468: 1833, 1833, 1833, 474: 1833, 476: 1833, 1833, 479: 1833, 1833, 1833, 1833, 1833, 485: 1833, 487: 1833, 1833, 1833, 1833, 492: 1833, 494: 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 508: 1833, 510: 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 531: 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833, 1833}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 2497, 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3764, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 2495, 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 632: 2491, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3763, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 738: 3766, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 3768, 3767, 3765, 761: 3993}, + {1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 460: 1834, 1834, 465: 1834, 1834, 468: 1834, 1834, 1834, 474: 1834, 476: 1834, 1834, 479: 1834, 1834, 1834, 1834, 1834, 485: 1834, 487: 1834, 1834, 1834, 1834, 492: 1834, 494: 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 508: 1834, 510: 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 531: 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834, 1834}, + {7: 3477, 57: 3994}, // 1545 - {1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 460: 1835, 1835, 465: 1835, 1835, 468: 1835, 1835, 1835, 474: 1835, 476: 1835, 1835, 479: 1835, 1835, 1835, 1835, 1835, 485: 1835, 487: 1835, 1835, 1835, 1835, 1835, 494: 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 508: 1835, 510: 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 531: 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835}, - {2: 1847, 1847, 1847, 1847, 1847, 8: 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 58: 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 459: 1847, 461: 1847, 1847, 464: 1847, 469: 1847, 1847, 1847, 1847, 1847, 478: 1847, 484: 1847, 486: 1847, 492: 1847, 1847, 529: 1847, 552: 1847, 554: 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 564: 1847, 1847, 1847, 1847, 1847, 1847, 572: 1847, 574: 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847}, + {1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 460: 1835, 1835, 465: 1835, 1835, 468: 1835, 1835, 1835, 474: 1835, 476: 1835, 1835, 479: 1835, 1835, 1835, 1835, 1835, 485: 1835, 487: 1835, 1835, 1835, 1835, 492: 1835, 494: 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 508: 1835, 510: 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 531: 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835, 1835}, + {2: 1847, 1847, 1847, 1847, 1847, 8: 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 58: 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 459: 1847, 461: 1847, 1847, 464: 1847, 469: 1847, 1847, 1847, 1847, 1847, 478: 1847, 484: 1847, 486: 1847, 491: 1847, 493: 1847, 529: 1847, 552: 1847, 554: 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 564: 1847, 1847, 1847, 1847, 1847, 1847, 572: 1847, 574: 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847, 1847}, {459: 1843}, - {2: 1841, 1841, 1841, 1841, 1841, 8: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 58: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 459: 1841, 461: 1841, 1841, 464: 1841, 469: 1841, 1841, 1841, 1841, 1841, 478: 1841, 484: 1841, 486: 1841, 492: 1841, 1841, 529: 1841, 552: 1841, 554: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 564: 1841, 1841, 1841, 1841, 1841, 1841, 572: 1841, 574: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841}, - {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}, + {2: 1841, 1841, 1841, 1841, 1841, 8: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 58: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 459: 1841, 461: 1841, 1841, 464: 1841, 469: 1841, 1841, 1841, 1841, 1841, 478: 1841, 484: 1841, 486: 1841, 491: 1841, 493: 1841, 529: 1841, 552: 1841, 554: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 564: 1841, 1841, 1841, 1841, 1841, 1841, 572: 1841, 574: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841}, + {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, 491: 1839, 493: 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, 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}, + {167: 4022, 478: 4023, 557: 4021, 4020}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 4014, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 4015, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 4013, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 638: 4016, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 4011, 1150: 4012}, + {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, 491: 1856, 493: 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, 491: 1855, 493: 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, 491: 1854, 493: 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}, // 1555 - {2: 1853, 1853, 1853, 1853, 1853, 8: 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 58: 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 459: 1853, 461: 1853, 1853, 464: 1853, 469: 1853, 1853, 1853, 1853, 1853, 478: 1853, 484: 1853, 486: 1853, 492: 1853, 1853, 529: 1853, 552: 1853, 554: 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 564: 1853, 1853, 1853, 1853, 1853, 1853, 572: 1853, 574: 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 638: 1853}, - {2: 1852, 1852, 1852, 1852, 1852, 8: 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 58: 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 459: 1852, 461: 1852, 1852, 464: 1852, 469: 1852, 1852, 1852, 1852, 1852, 478: 1852, 484: 1852, 486: 1852, 492: 1852, 1852, 529: 1852, 552: 1852, 554: 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 564: 1852, 1852, 1852, 1852, 1852, 1852, 572: 1852, 574: 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 638: 1852}, - {2: 1851, 1851, 1851, 1851, 1851, 8: 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 58: 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 459: 1851, 461: 1851, 1851, 464: 1851, 469: 1851, 1851, 1851, 1851, 1851, 478: 1851, 484: 1851, 486: 1851, 492: 1851, 1851, 529: 1851, 552: 1851, 554: 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 564: 1851, 1851, 1851, 1851, 1851, 1851, 572: 1851, 574: 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 638: 1851}, - {2: 1850, 1850, 1850, 1850, 1850, 8: 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 58: 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 459: 1850, 461: 1850, 1850, 464: 1850, 469: 1850, 1850, 1850, 1850, 1850, 478: 1850, 484: 1850, 486: 1850, 492: 1850, 1850, 529: 1850, 552: 1850, 554: 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 564: 1850, 1850, 1850, 1850, 1850, 1850, 572: 1850, 574: 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 638: 1850}, - {2: 1849, 1849, 1849, 1849, 1849, 8: 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 58: 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 459: 1849, 461: 1849, 1849, 464: 1849, 469: 1849, 1849, 1849, 1849, 1849, 478: 1849, 484: 1849, 486: 1849, 492: 1849, 1849, 529: 1849, 552: 1849, 554: 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 564: 1849, 1849, 1849, 1849, 1849, 1849, 572: 1849, 574: 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 638: 1849}, + {2: 1853, 1853, 1853, 1853, 1853, 8: 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 58: 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 459: 1853, 461: 1853, 1853, 464: 1853, 469: 1853, 1853, 1853, 1853, 1853, 478: 1853, 484: 1853, 486: 1853, 491: 1853, 493: 1853, 529: 1853, 552: 1853, 554: 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 564: 1853, 1853, 1853, 1853, 1853, 1853, 572: 1853, 574: 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 1853, 638: 1853}, + {2: 1852, 1852, 1852, 1852, 1852, 8: 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 58: 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 459: 1852, 461: 1852, 1852, 464: 1852, 469: 1852, 1852, 1852, 1852, 1852, 478: 1852, 484: 1852, 486: 1852, 491: 1852, 493: 1852, 529: 1852, 552: 1852, 554: 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 564: 1852, 1852, 1852, 1852, 1852, 1852, 572: 1852, 574: 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 638: 1852}, + {2: 1851, 1851, 1851, 1851, 1851, 8: 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 58: 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 459: 1851, 461: 1851, 1851, 464: 1851, 469: 1851, 1851, 1851, 1851, 1851, 478: 1851, 484: 1851, 486: 1851, 491: 1851, 493: 1851, 529: 1851, 552: 1851, 554: 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 564: 1851, 1851, 1851, 1851, 1851, 1851, 572: 1851, 574: 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 638: 1851}, + {2: 1850, 1850, 1850, 1850, 1850, 8: 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 58: 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 459: 1850, 461: 1850, 1850, 464: 1850, 469: 1850, 1850, 1850, 1850, 1850, 478: 1850, 484: 1850, 486: 1850, 491: 1850, 493: 1850, 529: 1850, 552: 1850, 554: 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 564: 1850, 1850, 1850, 1850, 1850, 1850, 572: 1850, 574: 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 638: 1850}, + {2: 1849, 1849, 1849, 1849, 1849, 8: 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 58: 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 459: 1849, 461: 1849, 1849, 464: 1849, 469: 1849, 1849, 1849, 1849, 1849, 478: 1849, 484: 1849, 486: 1849, 491: 1849, 493: 1849, 529: 1849, 552: 1849, 554: 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 564: 1849, 1849, 1849, 1849, 1849, 1849, 572: 1849, 574: 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 638: 1849}, // 1560 - {166: 1846, 462: 3970, 3969, 478: 1846, 557: 1846, 1846, 796: 4009}, - {166: 1845, 478: 1845, 557: 1845, 1845}, - {1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 460: 1860, 1860, 465: 1860, 1860, 468: 1860, 1860, 1860, 474: 1860, 476: 1860, 1860, 479: 1860, 1860, 1860, 1860, 1860, 485: 1860, 487: 1860, 1860, 1860, 1860, 1860, 494: 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 508: 1860, 510: 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 531: 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860}, - {459: 2497, 696: 4018}, - {715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 460: 715, 715, 715, 715, 465: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 476: 715, 715, 479: 715, 715, 715, 715, 715, 485: 715, 487: 715, 715, 715, 715, 715, 494: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 530: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 570: 715, 647: 4016}, + {167: 1846, 462: 3971, 3970, 478: 1846, 557: 1846, 1846, 796: 4010}, + {167: 1845, 478: 1845, 557: 1845, 1845}, + {1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 460: 1860, 1860, 465: 1860, 1860, 468: 1860, 1860, 1860, 474: 1860, 476: 1860, 1860, 479: 1860, 1860, 1860, 1860, 1860, 485: 1860, 487: 1860, 1860, 1860, 1860, 492: 1860, 494: 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 508: 1860, 510: 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 531: 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860}, + {459: 2498, 696: 4019}, + {715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 460: 715, 715, 715, 715, 465: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 476: 715, 715, 479: 715, 715, 715, 715, 715, 485: 715, 487: 715, 715, 715, 715, 492: 715, 494: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 530: 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 715, 570: 715, 647: 4017}, // 1565 - {1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1838, 1662, 1662, 1662, 1662, 465: 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 476: 1662, 1662, 479: 1662, 1662, 1662, 1662, 1662, 485: 1662, 487: 1662, 1662, 1662, 1662, 1662, 494: 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 530: 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 570: 1662, 639: 1662, 642: 1662, 1662}, - {1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1837, 1661, 1661, 1661, 1661, 465: 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 476: 1661, 1661, 479: 1661, 1661, 1661, 1661, 1661, 485: 1661, 487: 1661, 1661, 1661, 1661, 1661, 494: 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 530: 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 570: 1661, 639: 1661, 642: 1661, 1661}, + {1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1838, 1662, 1662, 1662, 1662, 465: 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 476: 1662, 1662, 479: 1662, 1662, 1662, 1662, 1662, 485: 1662, 487: 1662, 1662, 1662, 1662, 492: 1662, 494: 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 530: 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 570: 1662, 639: 1662, 642: 1662, 1662}, + {1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1837, 1661, 1661, 1661, 1661, 465: 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 476: 1661, 1661, 479: 1661, 1661, 1661, 1661, 1661, 485: 1661, 487: 1661, 1661, 1661, 1661, 492: 1661, 494: 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 530: 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 570: 1661, 639: 1661, 642: 1661, 1661}, {459: 1836}, - {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, 2661, 4017}, - {1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 460: 1858, 1858, 465: 1858, 1858, 468: 1858, 1858, 1858, 474: 1858, 476: 1858, 1858, 479: 1858, 1858, 1858, 1858, 1858, 485: 1858, 487: 1858, 1858, 1858, 1858, 1858, 494: 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 508: 1858, 510: 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 531: 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 4018}, + {1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 460: 1858, 1858, 465: 1858, 1858, 468: 1858, 1858, 1858, 474: 1858, 476: 1858, 1858, 479: 1858, 1858, 1858, 1858, 1858, 485: 1858, 487: 1858, 1858, 1858, 1858, 492: 1858, 494: 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 508: 1858, 510: 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 531: 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858}, // 1570 - {1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 460: 1859, 1859, 465: 1859, 1859, 468: 1859, 1859, 1859, 474: 1859, 476: 1859, 1859, 479: 1859, 1859, 1859, 1859, 1859, 485: 1859, 487: 1859, 1859, 1859, 1859, 1859, 494: 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 508: 1859, 510: 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 531: 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859}, - {1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 460: 1885, 1885, 465: 1885, 1885, 468: 1885, 1885, 1885, 474: 1885, 476: 1885, 1885, 479: 1885, 1885, 1885, 483: 1885, 485: 1885, 487: 1885, 1885, 1885, 1885, 1885, 494: 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 508: 1885, 510: 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 531: 1885, 1885}, - {1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 460: 1884, 1884, 465: 1884, 1884, 468: 1884, 1884, 1884, 474: 1884, 476: 1884, 1884, 479: 1884, 1884, 1884, 483: 1884, 485: 1884, 487: 1884, 1884, 1884, 1884, 1884, 494: 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 508: 1884, 510: 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 531: 1884, 1884}, - {1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 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}, + {1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 460: 1859, 1859, 465: 1859, 1859, 468: 1859, 1859, 1859, 474: 1859, 476: 1859, 1859, 479: 1859, 1859, 1859, 1859, 1859, 485: 1859, 487: 1859, 1859, 1859, 1859, 492: 1859, 494: 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 508: 1859, 510: 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 531: 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859}, + {1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 460: 1885, 1885, 465: 1885, 1885, 468: 1885, 1885, 1885, 474: 1885, 476: 1885, 1885, 479: 1885, 1885, 1885, 483: 1885, 485: 1885, 487: 1885, 1885, 1885, 1885, 492: 1885, 494: 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 508: 1885, 510: 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 1885, 531: 1885, 1885}, + {1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 460: 1884, 1884, 465: 1884, 1884, 468: 1884, 1884, 1884, 474: 1884, 476: 1884, 1884, 479: 1884, 1884, 1884, 483: 1884, 485: 1884, 487: 1884, 1884, 1884, 1884, 492: 1884, 494: 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 508: 1884, 510: 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 531: 1884, 1884}, + {1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 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, 492: 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, 492: 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, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4026, 795: 4027}, + {2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 459: 2256, 475: 2256, 482: 2256, 486: 2256, 498: 2256, 515: 2256, 2256, 529: 2256, 634: 2256, 639: 4048, 656: 2256, 2256, 659: 2256, 664: 2256, 2256, 667: 2256, 2256, 2256, 2256, 2256, 2256, 675: 2256, 677: 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 2256, 692: 2256, 2256, 2256, 2256}, + {7: 2253, 57: 2253}, + {7: 4028, 57: 4029}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4047}, // 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, 1195: 4032}, - {57: 4045}, + {291: 4030}, + {459: 4031}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 4032}, + {57: 1879, 460: 4035, 471: 3574, 3573, 3579, 509: 3575, 530: 4034, 542: 3576, 3577, 3570, 3580, 3569, 3578, 3571, 3572, 1195: 4033}, + {57: 4046}, // 1585 - {218: 4038, 504: 4037}, - {143: 4035}, - {240: 4036}, + {219: 4039, 504: 4038}, + {143: 4036}, + {241: 4037}, {57: 1875}, - {335: 4040}, + {335: 4041}, // 1590 - {201: 4039}, + {201: 4040}, {57: 1876}, - {201: 4041}, - {57: 1878, 460: 4042}, - {143: 4043}, + {201: 4042}, + {57: 1878, 460: 4043}, + {143: 4044}, // 1595 - {240: 4044}, + {241: 4045}, {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}, - {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}, + {1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 1886, 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, 492: 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}, + {7: 2252, 57: 2252}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4049, 2665, 2666, 2664}, // 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, 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, 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}, + {2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 2255, 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: 4050, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4051, 2665, 2666, 2664}, + {2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 2254, 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, 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}, + {1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 1887, 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, 492: 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: 3231, 3229}, {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 {1263, 1263, 7: 1263, 57: 1263, 124: 1263, 458: 1263, 460: 1263, 466: 1263, 468: 1263, 476: 1263, 1263, 479: 1263, 1263, 1263, 483: 1263, 488: 1263, 490: 1263, 501: 1263, 1263, 510: 1263, 513: 1263, 1263}, {1262, 1262, 7: 1262, 57: 1262, 124: 1262, 458: 1262, 460: 1262, 466: 1262, 468: 1262, 476: 1262, 1262, 479: 1262, 1262, 1262, 483: 1262, 488: 1262, 490: 1262, 501: 1262, 1262, 510: 1262, 513: 1262, 1262}, - {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, 4056}, - {1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 460: 1891, 1891, 465: 1891, 1891, 468: 1891, 1891, 1891, 474: 1891, 476: 1891, 1891, 479: 1891, 1891, 1891, 483: 1891, 485: 1891, 487: 1891, 1891, 1891, 1891, 3234, 494: 3232, 3233, 3231, 3229, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 508: 1891, 510: 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 531: 1891, 1891, 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, 648: 3805, 2664, 2665, 2663, 724: 4058}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 4057}, + {1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 460: 1891, 1891, 465: 1891, 1891, 468: 1891, 1891, 1891, 474: 1891, 476: 1891, 1891, 479: 1891, 1891, 1891, 483: 1891, 485: 1891, 487: 1891, 1891, 1891, 1891, 492: 3235, 494: 3233, 3234, 3232, 3230, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 508: 1891, 510: 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 1891, 531: 1891, 1891, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 4059}, // 1610 - {57: 4059}, - {2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 460: 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 485: 2160, 487: 2160, 2160, 2160, 2160, 2160, 494: 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 530: 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 563: 2160, 570: 2160, 2160, 573: 2160, 629: 2160, 2160, 2160, 633: 2160}, - {479: 4061}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4062}, - {2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 460: 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 485: 2161, 487: 2161, 2161, 2161, 2161, 2161, 494: 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 530: 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 563: 2161, 570: 2161, 2161, 573: 2161, 629: 2161, 2161, 2161, 633: 2161}, + {57: 4060}, + {2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 460: 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 485: 2161, 487: 2161, 2161, 2161, 2161, 492: 2161, 494: 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 530: 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 563: 2161, 570: 2161, 2161, 573: 2161, 629: 2161, 2161, 2161, 633: 2161}, + {479: 4062}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 4063}, + {2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 460: 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 485: 2162, 487: 2162, 2162, 2162, 2162, 492: 2162, 494: 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 530: 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 563: 2162, 570: 2162, 2162, 573: 2162, 629: 2162, 2162, 2162, 633: 2162}, // 1615 - {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}, + {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, 492: 3235, 494: 3233, 3234, 3232, 3230, 499: 238, 501: 238, 238, 720: 3231, 3229}, {4, 4}, - {143: 4066}, - {237, 237, 480: 237, 488: 237, 2624, 237, 777: 2625, 4067}, - {1258, 1258, 480: 1258, 488: 1258, 490: 2627, 753: 2628, 797: 4068}, + {143: 4067}, + {237, 237, 480: 237, 488: 237, 2625, 237, 777: 2626, 4068}, + {1258, 1258, 480: 1258, 488: 1258, 490: 2628, 753: 2629, 797: 4069}, // 1620 - {834, 834, 480: 2630, 488: 2631, 754: 2632, 815: 4069}, + {834, 834, 480: 2631, 488: 2632, 754: 2633, 815: 4070}, {2, 2}, - {553: 4072}, + {553: 4073}, {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 4074}, // 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, 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}, + {2330, 2330, 2330, 2330, 4132, 4134, 389, 13: 2107, 4151, 4078, 4090, 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 4149, 4169, 4153, 4140, 4133, 4136, 4135, 4138, 4139, 4141, 4148, 389, 4159, 4160, 4146, 4147, 4152, 4154, 4166, 4165, 4171, 4167, 4164, 4157, 4162, 4163, 4156, 4158, 4161, 4150, 80: 4103, 83: 4124, 4125, 92: 4126, 132: 4106, 190: 4091, 4110, 194: 4111, 207: 4105, 214: 4121, 225: 4100, 235: 4107, 239: 4102, 254: 4112, 262: 4108, 269: 4122, 4123, 276: 4092, 460: 4120, 464: 4131, 466: 4168, 2107, 475: 2330, 483: 4127, 485: 4119, 2107, 490: 4109, 498: 4094, 571: 4099, 4095, 634: 2107, 4137, 640: 4076, 653: 4114, 656: 4101, 658: 4128, 666: 4113, 673: 4115, 676: 4096, 691: 4104, 758: 4088, 762: 4089, 766: 4142, 779: 4144, 798: 4143, 821: 4145, 825: 4155, 829: 4170, 850: 4118, 865: 4116, 904: 4093, 910: 4097, 971: 4130, 1115: 4098, 1142: 4117, 1147: 4129, 4075}, + {2105, 2105, 4913, 4914, 475: 4915, 1078: 4912, 1146: 4911}, + {475: 4885}, + {461: 1988, 482: 4172, 723: 4883}, + {461: 1988, 482: 4172, 723: 4881}, // 1630 - {482: 4171, 493: 1987, 723: 4878}, - {482: 4171, 493: 1987, 723: 4876}, - {482: 4171, 493: 1987, 723: 4874}, - {461: 1987, 482: 4171, 723: 4872}, - {461: 1987, 482: 4171, 723: 4870}, + {482: 4172, 493: 1988, 723: 4879}, + {482: 4172, 493: 1988, 723: 4877}, + {482: 4172, 493: 1988, 723: 4875}, + {461: 1988, 482: 4172, 723: 4873}, + {461: 1988, 482: 4172, 723: 4871}, // 1635 - {461: 1987, 482: 4171, 723: 4868}, - {461: 1987, 482: 4171, 723: 4866}, - {461: 1987, 482: 4171, 723: 4864}, - {461: 1987, 482: 4171, 723: 4862}, - {2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 13: 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 458: 2429, 2429, 2429, 464: 2429, 2429, 2429, 2429, 474: 2429, 2429, 484: 2429, 2429, 2429, 492: 2429, 553: 2429, 632: 2429, 634: 2429, 2429}, + {461: 1988, 482: 4172, 723: 4869}, + {461: 1988, 482: 4172, 723: 4867}, + {461: 1988, 482: 4172, 723: 4865}, + {461: 1988, 482: 4172, 723: 4863}, + {2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 13: 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 458: 2430, 2430, 2430, 464: 2430, 2430, 2430, 2430, 474: 2430, 2430, 484: 2430, 2430, 2430, 491: 2430, 553: 2430, 632: 2430, 634: 2430, 2430}, // 1640 - {2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 13: 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 458: 2428, 2428, 2428, 464: 2428, 2428, 2428, 2428, 474: 2428, 2428, 484: 2428, 2428, 2428, 492: 2428, 553: 2428, 632: 2428, 634: 2428, 2428}, - {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, 779: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4845}, + {2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 13: 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 2429, 458: 2429, 2429, 2429, 464: 2429, 2429, 2429, 2429, 474: 2429, 2429, 484: 2429, 2429, 2429, 491: 2429, 553: 2429, 632: 2429, 634: 2429, 2429}, + {155: 4855}, + {461: 1988, 464: 1988, 482: 4172, 723: 4852}, + {461: 1988, 464: 1988, 482: 4172, 723: 4849}, + {2413, 2413, 2413, 2413, 4132, 4134, 389, 2413, 13: 2107, 4151, 4078, 4090, 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 4149, 4169, 4153, 4140, 4133, 4136, 4135, 4138, 4139, 4141, 4148, 389, 4159, 4160, 4146, 4147, 4152, 4154, 4166, 4165, 4171, 4167, 4164, 4157, 4162, 4163, 4156, 4158, 4161, 4150, 464: 4131, 466: 4168, 2107, 475: 2413, 485: 4845, 2107, 634: 2107, 4137, 758: 4088, 762: 4089, 766: 4142, 779: 4144, 798: 4143, 821: 4145, 825: 4155, 829: 4846}, // 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, 885: 4684, 899: 4686}, + {382: 4835}, + {637: 4827}, + {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, 475: 4686, 552: 2335, 563: 2324, 571: 2324, 573: 2324, 629: 2324, 4479, 635: 2324, 662: 2324, 2324, 819: 4688, 831: 4314, 854: 4684, 888: 4685, 899: 4687}, + {2403, 2403, 2403, 2403, 7: 2403, 475: 2403}, {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, 899: 4658, 981: 4661}, - {475: 4644}, - {475: 4642}, + {475: 4682}, + {475: 4679}, + {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, 475: 4658, 552: 2335, 563: 4300, 571: 4315, 573: 4657, 630: 4316, 635: 4301, 662: 4661, 814: 4660, 831: 4314, 854: 4656, 899: 4659, 982: 4662}, + {475: 4645}, + {475: 4643}, // 1655 - {475: 4639}, - {475: 4636}, - {30: 4633, 475: 4632}, - {30: 4629, 475: 4628}, - {475: 4618}, + {475: 4640}, + {475: 4637}, + {30: 4634, 475: 4633}, + {30: 4630, 475: 4629}, + {475: 4619}, // 1660 - {646: 4611}, + {646: 4612}, + {927: 4611}, {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}, + {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, 552: 2335, 831: 4314, 854: 4606}, + {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, 552: 2335, 831: 4314, 854: 4340}, // 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, 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}, + {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, 571: 4315, 630: 4316, 635: 4313, 831: 4314, 854: 4311, 982: 4312}, + {2: 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, 465: 4298, 482: 4172, 563: 4300, 635: 4301, 637: 4296, 723: 4297, 814: 4299, 831: 4295}, + {2371, 2371, 2371, 2371, 7: 2371, 475: 2371}, {2370, 2370, 2370, 2370, 7: 2370, 475: 2370}, {2369, 2369, 2369, 2369, 7: 2369, 475: 2369}, - {2368, 2368, 2368, 2368, 7: 2368, 475: 2368}, // 1670 - {2367, 2367, 2367, 2367, 6: 388, 2367, 38: 388, 475: 2367}, + {2368, 2368, 2368, 2368, 6: 388, 2368, 38: 388, 475: 2368}, + {188: 4294}, {188: 4293}, - {188: 4292}, + {2365, 2365, 2365, 2365, 7: 2365, 475: 2365}, {2364, 2364, 2364, 2364, 7: 2364, 475: 2364}, - {2363, 2363, 2363, 2363, 7: 2363, 475: 2363}, // 1675 + {2360, 2360, 2360, 2360, 7: 2360, 475: 2360}, {2359, 2359, 2359, 2359, 7: 2359, 475: 2359}, - {2358, 2358, 2358, 2358, 7: 2358, 475: 2358}, - {161: 1987, 228: 1987, 246: 1987, 1987, 464: 1987, 482: 4171, 723: 4286}, - {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, 464: 1987, 482: 4171, 723: 4283}, - {149: 4282, 674: 4281}, + {163: 1988, 229: 1988, 247: 1988, 1988, 464: 1988, 482: 4172, 723: 4287}, + {2: 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, 464: 1988, 482: 4172, 723: 4284}, + {149: 4283, 674: 4282}, // 1680 - {2328, 2328, 2328, 2328, 7: 4279, 475: 2328}, - {2327, 2327, 2327, 2327, 7: 2327, 475: 2327}, - {13: 2105, 26: 2105, 28: 2105, 467: 2105, 486: 2105, 634: 2105}, - {461: 1987, 482: 4171, 723: 4277}, - {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, 723: 4275}, + {2329, 2329, 2329, 2329, 7: 4280, 475: 2329}, + {2328, 2328, 2328, 2328, 7: 2328, 475: 2328}, + {13: 2106, 16: 2106, 28: 2106, 467: 2106, 486: 2106, 634: 2106}, + {461: 1988, 482: 4172, 723: 4278}, + {2: 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, 461: 1988, 482: 4172, 723: 4276}, // 1685 - {31: 4270, 176: 4271, 235: 4272}, - {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, 723: 4268}, - {233: 4265}, - {233: 4262}, - {482: 4171, 493: 1987, 723: 4260}, + {31: 4271, 176: 4272, 236: 4273}, + {2: 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, 461: 1988, 482: 4172, 723: 4269}, + {234: 4266}, + {234: 4263}, + {482: 4172, 493: 1988, 723: 4261}, // 1690 - {482: 4171, 493: 1987, 723: 4258}, - {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, 482: 4171, 723: 4256}, - {482: 4171, 493: 1987, 723: 4254}, - {2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 13: 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 458: 2060, 2060, 2060, 464: 2060, 2060, 2060, 2060, 474: 2060, 2060, 484: 2060, 2060, 2060, 492: 2060, 553: 2060, 632: 2060, 634: 2060, 2060}, - {423, 423, 423, 423, 423, 423, 423, 423, 13: 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 458: 423, 423, 423, 464: 423, 423, 423, 423, 474: 423, 423, 484: 423, 423, 423, 492: 423, 553: 423, 632: 423, 634: 423, 423}, + {482: 4172, 493: 1988, 723: 4259}, + {2: 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, 482: 4172, 723: 4257}, + {482: 4172, 493: 1988, 723: 4255}, + {2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 13: 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 458: 2061, 2061, 2061, 464: 2061, 2061, 2061, 2061, 474: 2061, 2061, 484: 2061, 2061, 2061, 491: 2061, 553: 2061, 632: 2061, 634: 2061, 2061}, + {423, 423, 423, 423, 423, 423, 423, 423, 13: 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 458: 423, 423, 423, 464: 423, 423, 423, 423, 474: 423, 423, 484: 423, 423, 423, 491: 423, 553: 423, 632: 423, 634: 423, 423}, // 1695 - {13: 3720, 467: 4249, 486: 3721, 634: 3719, 759: 4248}, - {6: 4242, 38: 4243}, - {482: 4171, 493: 1987, 723: 4240}, - {482: 4171, 493: 1987, 723: 4238}, - {461: 1987, 482: 4171, 723: 4236}, + {13: 3721, 467: 4250, 486: 3722, 634: 3720, 759: 4249}, + {6: 4243, 38: 4244}, + {482: 4172, 493: 1988, 723: 4241}, + {482: 4172, 493: 1988, 723: 4239}, + {461: 1988, 482: 4172, 723: 4237}, // 1700 - {482: 4171, 493: 1987, 723: 4234}, - {482: 4171, 493: 1987, 723: 4232}, - {461: 1987, 482: 4171, 723: 4230}, - {461: 1987, 482: 4171, 723: 4228}, - {482: 4171, 493: 1987, 723: 4226}, + {482: 4172, 493: 1988, 723: 4235}, + {482: 4172, 493: 1988, 723: 4233}, + {461: 1988, 482: 4172, 723: 4231}, + {461: 1988, 482: 4172, 723: 4229}, + {482: 4172, 493: 1988, 723: 4227}, // 1705 - {482: 4171, 493: 1987, 723: 4224}, - {409, 409, 409, 409, 409, 409, 409, 409, 13: 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 458: 409, 409, 409, 464: 409, 409, 409, 409, 474: 409, 409, 484: 409, 409, 409, 492: 409, 553: 409, 632: 409, 634: 409, 409}, - {464: 1987, 482: 4171, 493: 1987, 723: 4222}, - {464: 1987, 482: 4171, 493: 1987, 723: 4219}, - {464: 1987, 482: 4171, 493: 1987, 723: 4216}, + {482: 4172, 493: 1988, 723: 4225}, + {409, 409, 409, 409, 409, 409, 409, 409, 13: 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 458: 409, 409, 409, 464: 409, 409, 409, 409, 474: 409, 409, 484: 409, 409, 409, 491: 409, 553: 409, 632: 409, 634: 409, 409}, + {464: 1988, 482: 4172, 493: 1988, 723: 4223}, + {464: 1988, 482: 4172, 493: 1988, 723: 4220}, + {464: 1988, 482: 4172, 493: 1988, 723: 4217}, // 1710 - {482: 4171, 493: 1987, 723: 4214}, - {482: 4171, 493: 1987, 723: 4212}, - {482: 4171, 493: 1987, 559: 1987, 1987, 723: 4210}, - {461: 1987, 482: 4171, 723: 4208}, - {461: 1987, 482: 4171, 723: 4206}, + {482: 4172, 493: 1988, 723: 4215}, + {482: 4172, 493: 1988, 723: 4213}, + {482: 4172, 493: 1988, 559: 1988, 1988, 723: 4211}, + {461: 1988, 482: 4172, 723: 4209}, + {461: 1988, 482: 4172, 723: 4207}, // 1715 - {482: 4171, 493: 1987, 723: 4204}, - {482: 4171, 493: 1987, 723: 4202}, - {464: 1987, 482: 4171, 493: 1987, 723: 4198}, - {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, 478: 1987, 482: 4171, 723: 4195}, - {459: 1987, 482: 4171, 723: 4190}, + {482: 4172, 493: 1988, 723: 4205}, + {482: 4172, 493: 1988, 723: 4203}, + {464: 1988, 482: 4172, 493: 1988, 723: 4199}, + {2: 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, 461: 1988, 478: 1988, 482: 4172, 723: 4196}, + {459: 1988, 482: 4172, 723: 4191}, // 1720 - {461: 1987, 482: 4171, 723: 4187}, - {383, 383, 383, 383, 383, 383, 383, 383, 13: 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 458: 383, 383, 383, 464: 383, 383, 383, 383, 474: 383, 383, 484: 383, 383, 383, 492: 383, 553: 383, 632: 383, 634: 383, 383}, - {172: 1987, 193: 1987, 225: 1987, 1987, 263: 1987, 279: 1987, 1987, 1987, 1987, 1987, 1987, 1987, 1987, 464: 1987, 482: 4171, 723: 4172}, - {2: 1986, 1986, 1986, 1986, 1986, 8: 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 58: 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 1986, 459: 1986, 461: 1986, 464: 1986, 471: 1986, 1986, 474: 1986, 478: 1986, 492: 1986, 1986, 529: 1986, 557: 1986, 1986, 1986, 1986}, - {172: 4175, 193: 4174, 225: 4178, 4176, 263: 4177, 279: 4179, 4180, 4184, 4183, 4181, 4185, 4186, 4182, 464: 4173}, + {461: 1988, 482: 4172, 723: 4188}, + {383, 383, 383, 383, 383, 383, 383, 383, 13: 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 458: 383, 383, 383, 464: 383, 383, 383, 383, 474: 383, 383, 484: 383, 383, 383, 491: 383, 553: 383, 632: 383, 634: 383, 383}, + {172: 1988, 193: 1988, 226: 1988, 1988, 263: 1988, 279: 1988, 1988, 1988, 1988, 1988, 1988, 1988, 1988, 464: 1988, 482: 4172, 723: 4173}, + {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, 459: 1987, 461: 1987, 464: 1987, 471: 1987, 1987, 474: 1987, 478: 1987, 491: 1987, 493: 1987, 529: 1987, 557: 1987, 1987, 1987, 1987}, + {172: 4176, 193: 4175, 226: 4179, 4177, 263: 4178, 279: 4180, 4181, 4185, 4184, 4182, 4186, 4187, 4183, 464: 4174}, // 1725 - {377, 377, 377, 377, 377, 377, 377, 377, 13: 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 458: 377, 377, 377, 464: 377, 377, 377, 377, 474: 377, 377, 484: 377, 377, 377, 492: 377, 553: 377, 632: 377, 634: 377, 377}, - {376, 376, 376, 376, 376, 376, 376, 376, 13: 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 458: 376, 376, 376, 464: 376, 376, 376, 376, 474: 376, 376, 484: 376, 376, 376, 492: 376, 553: 376, 632: 376, 634: 376, 376}, - {375, 375, 375, 375, 375, 375, 375, 375, 13: 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 458: 375, 375, 375, 464: 375, 375, 375, 375, 474: 375, 375, 484: 375, 375, 375, 492: 375, 553: 375, 632: 375, 634: 375, 375}, - {374, 374, 374, 374, 374, 374, 374, 374, 13: 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 458: 374, 374, 374, 464: 374, 374, 374, 374, 474: 374, 374, 484: 374, 374, 374, 492: 374, 553: 374, 632: 374, 634: 374, 374}, - {373, 373, 373, 373, 373, 373, 373, 373, 13: 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 458: 373, 373, 373, 464: 373, 373, 373, 373, 474: 373, 373, 484: 373, 373, 373, 492: 373, 553: 373, 632: 373, 634: 373, 373}, + {377, 377, 377, 377, 377, 377, 377, 377, 13: 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 458: 377, 377, 377, 464: 377, 377, 377, 377, 474: 377, 377, 484: 377, 377, 377, 491: 377, 553: 377, 632: 377, 634: 377, 377}, + {376, 376, 376, 376, 376, 376, 376, 376, 13: 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 458: 376, 376, 376, 464: 376, 376, 376, 376, 474: 376, 376, 484: 376, 376, 376, 491: 376, 553: 376, 632: 376, 634: 376, 376}, + {375, 375, 375, 375, 375, 375, 375, 375, 13: 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 458: 375, 375, 375, 464: 375, 375, 375, 375, 474: 375, 375, 484: 375, 375, 375, 491: 375, 553: 375, 632: 375, 634: 375, 375}, + {374, 374, 374, 374, 374, 374, 374, 374, 13: 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 458: 374, 374, 374, 464: 374, 374, 374, 374, 474: 374, 374, 484: 374, 374, 374, 491: 374, 553: 374, 632: 374, 634: 374, 374}, + {373, 373, 373, 373, 373, 373, 373, 373, 13: 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 458: 373, 373, 373, 464: 373, 373, 373, 373, 474: 373, 373, 484: 373, 373, 373, 491: 373, 553: 373, 632: 373, 634: 373, 373}, // 1730 - {372, 372, 372, 372, 372, 372, 372, 372, 13: 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 458: 372, 372, 372, 464: 372, 372, 372, 372, 474: 372, 372, 484: 372, 372, 372, 492: 372, 553: 372, 632: 372, 634: 372, 372}, - {371, 371, 371, 371, 371, 371, 371, 371, 13: 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 458: 371, 371, 371, 464: 371, 371, 371, 371, 474: 371, 371, 484: 371, 371, 371, 492: 371, 553: 371, 632: 371, 634: 371, 371}, - {370, 370, 370, 370, 370, 370, 370, 370, 13: 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 458: 370, 370, 370, 464: 370, 370, 370, 370, 474: 370, 370, 484: 370, 370, 370, 492: 370, 553: 370, 632: 370, 634: 370, 370}, - {369, 369, 369, 369, 369, 369, 369, 369, 13: 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 458: 369, 369, 369, 464: 369, 369, 369, 369, 474: 369, 369, 484: 369, 369, 369, 492: 369, 553: 369, 632: 369, 634: 369, 369}, - {368, 368, 368, 368, 368, 368, 368, 368, 13: 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 458: 368, 368, 368, 464: 368, 368, 368, 368, 474: 368, 368, 484: 368, 368, 368, 492: 368, 553: 368, 632: 368, 634: 368, 368}, + {372, 372, 372, 372, 372, 372, 372, 372, 13: 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 458: 372, 372, 372, 464: 372, 372, 372, 372, 474: 372, 372, 484: 372, 372, 372, 491: 372, 553: 372, 632: 372, 634: 372, 372}, + {371, 371, 371, 371, 371, 371, 371, 371, 13: 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 458: 371, 371, 371, 464: 371, 371, 371, 371, 474: 371, 371, 484: 371, 371, 371, 491: 371, 553: 371, 632: 371, 634: 371, 371}, + {370, 370, 370, 370, 370, 370, 370, 370, 13: 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 458: 370, 370, 370, 464: 370, 370, 370, 370, 474: 370, 370, 484: 370, 370, 370, 491: 370, 553: 370, 632: 370, 634: 370, 370}, + {369, 369, 369, 369, 369, 369, 369, 369, 13: 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 458: 369, 369, 369, 464: 369, 369, 369, 369, 474: 369, 369, 484: 369, 369, 369, 491: 369, 553: 369, 632: 369, 634: 369, 369}, + {368, 368, 368, 368, 368, 368, 368, 368, 13: 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 458: 368, 368, 368, 464: 368, 368, 368, 368, 474: 368, 368, 484: 368, 368, 368, 491: 368, 553: 368, 632: 368, 634: 368, 368}, // 1735 - {367, 367, 367, 367, 367, 367, 367, 367, 13: 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 458: 367, 367, 367, 464: 367, 367, 367, 367, 474: 367, 367, 484: 367, 367, 367, 492: 367, 553: 367, 632: 367, 634: 367, 367}, - {366, 366, 366, 366, 366, 366, 366, 366, 13: 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 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, 1020: 4188}, + {367, 367, 367, 367, 367, 367, 367, 367, 13: 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 458: 367, 367, 367, 464: 367, 367, 367, 367, 474: 367, 367, 484: 367, 367, 367, 491: 367, 553: 367, 632: 367, 634: 367, 367}, + {366, 366, 366, 366, 366, 366, 366, 366, 13: 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 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, 491: 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, 491: 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, 491: 364, 553: 364, 632: 364, 634: 364, 364}, + {461: 4190, 1021: 4189}, // 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, 782: 4192, 1123: 4193}, - {546, 546, 7: 3808, 57: 546, 460: 546}, + {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, 491: 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, 491: 9, 553: 9, 632: 9, 634: 9, 9}, + {459: 4192}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 547, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 3807, 782: 4193, 1124: 4194}, + {546, 546, 7: 3809, 57: 546, 460: 546}, // 1745 - {57: 4194}, - {391, 391, 391, 391, 391, 391, 391, 391, 13: 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 458: 391, 391, 391, 464: 391, 391, 391, 391, 474: 391, 391, 484: 391, 391, 391, 492: 391, 553: 391, 632: 391, 634: 391, 391}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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, 478: 4196, 648: 3377, 2664, 2665, 2663, 725: 4197}, - {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}, + {57: 4195}, + {391, 391, 391, 391, 391, 391, 391, 391, 13: 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 458: 391, 391, 391, 464: 391, 391, 391, 391, 474: 391, 391, 484: 391, 391, 391, 491: 391, 553: 391, 632: 391, 634: 391, 391}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 478: 4197, 648: 3378, 2665, 2666, 2664, 725: 4198}, + {393, 393, 393, 393, 393, 393, 393, 393, 13: 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 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, 491: 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, 491: 392, 553: 392, 632: 392, 634: 392, 392}, // 1750 - {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}, - {493: 2638, 722: 2637, 730: 4203}, + {464: 4201, 493: 2639, 722: 2638, 730: 4202, 1116: 4200}, + {396, 396, 396, 396, 396, 396, 396, 396, 13: 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 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, 491: 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, 491: 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, 491: 386, 553: 386, 632: 386, 634: 386, 386}, + {493: 2639, 722: 2638, 730: 4204}, // 1755 - {397, 397, 397, 397, 397, 397, 397, 397, 13: 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 458: 397, 397, 397, 464: 397, 397, 397, 397, 474: 397, 397, 484: 397, 397, 397, 492: 397, 553: 397, 632: 397, 634: 397, 397}, - {493: 2638, 722: 2637, 730: 4205}, - {398, 398, 398, 398, 398, 398, 398, 398, 13: 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 458: 398, 398, 398, 464: 398, 398, 398, 398, 474: 398, 398, 484: 398, 398, 398, 492: 398, 553: 398, 632: 398, 634: 398, 398}, - {461: 4207}, - {399, 399, 399, 399, 399, 399, 399, 399, 13: 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 458: 399, 399, 399, 464: 399, 399, 399, 399, 474: 399, 399, 484: 399, 399, 399, 492: 399, 553: 399, 632: 399, 634: 399, 399}, + {397, 397, 397, 397, 397, 397, 397, 397, 13: 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 458: 397, 397, 397, 464: 397, 397, 397, 397, 474: 397, 397, 484: 397, 397, 397, 491: 397, 553: 397, 632: 397, 634: 397, 397}, + {493: 2639, 722: 2638, 730: 4206}, + {398, 398, 398, 398, 398, 398, 398, 398, 13: 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 458: 398, 398, 398, 464: 398, 398, 398, 398, 474: 398, 398, 484: 398, 398, 398, 491: 398, 553: 398, 632: 398, 634: 398, 398}, + {461: 4208}, + {399, 399, 399, 399, 399, 399, 399, 399, 13: 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 458: 399, 399, 399, 464: 399, 399, 399, 399, 474: 399, 399, 484: 399, 399, 399, 491: 399, 553: 399, 632: 399, 634: 399, 399}, // 1760 - {461: 4209}, - {400, 400, 400, 400, 400, 400, 400, 400, 13: 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 458: 400, 400, 400, 464: 400, 400, 400, 400, 474: 400, 400, 484: 400, 400, 400, 492: 400, 553: 400, 632: 400, 634: 400, 400}, - {493: 3309, 559: 3311, 3310, 805: 4211}, - {401, 401, 401, 401, 401, 401, 401, 401, 13: 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 458: 401, 401, 401, 464: 401, 401, 401, 401, 474: 401, 401, 484: 401, 401, 401, 492: 401, 553: 401, 632: 401, 634: 401, 401}, - {493: 2638, 722: 2637, 730: 4213}, + {461: 4210}, + {400, 400, 400, 400, 400, 400, 400, 400, 13: 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 458: 400, 400, 400, 464: 400, 400, 400, 400, 474: 400, 400, 484: 400, 400, 400, 491: 400, 553: 400, 632: 400, 634: 400, 400}, + {493: 3310, 559: 3312, 3311, 806: 4212}, + {401, 401, 401, 401, 401, 401, 401, 401, 13: 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 458: 401, 401, 401, 464: 401, 401, 401, 401, 474: 401, 401, 484: 401, 401, 401, 491: 401, 553: 401, 632: 401, 634: 401, 401}, + {493: 2639, 722: 2638, 730: 4214}, // 1765 - {402, 402, 402, 402, 402, 402, 402, 402, 13: 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 458: 402, 402, 402, 464: 402, 402, 402, 402, 474: 402, 402, 484: 402, 402, 402, 492: 402, 553: 402, 632: 402, 634: 402, 402}, - {493: 2638, 722: 2637, 730: 4215}, - {403, 403, 403, 403, 403, 403, 403, 403, 13: 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 458: 403, 403, 403, 464: 403, 403, 403, 403, 474: 403, 403, 484: 403, 403, 403, 492: 403, 553: 403, 632: 403, 634: 403, 403}, - {464: 4218, 493: 2638, 722: 2637, 730: 4217}, - {405, 405, 405, 405, 405, 405, 405, 405, 13: 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 458: 405, 405, 405, 464: 405, 405, 405, 405, 474: 405, 405, 484: 405, 405, 405, 492: 405, 553: 405, 632: 405, 634: 405, 405}, + {402, 402, 402, 402, 402, 402, 402, 402, 13: 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 458: 402, 402, 402, 464: 402, 402, 402, 402, 474: 402, 402, 484: 402, 402, 402, 491: 402, 553: 402, 632: 402, 634: 402, 402}, + {493: 2639, 722: 2638, 730: 4216}, + {403, 403, 403, 403, 403, 403, 403, 403, 13: 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 458: 403, 403, 403, 464: 403, 403, 403, 403, 474: 403, 403, 484: 403, 403, 403, 491: 403, 553: 403, 632: 403, 634: 403, 403}, + {464: 4219, 493: 2639, 722: 2638, 730: 4218}, + {405, 405, 405, 405, 405, 405, 405, 405, 13: 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 458: 405, 405, 405, 464: 405, 405, 405, 405, 474: 405, 405, 484: 405, 405, 405, 491: 405, 553: 405, 632: 405, 634: 405, 405}, // 1770 - {404, 404, 404, 404, 404, 404, 404, 404, 13: 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 458: 404, 404, 404, 464: 404, 404, 404, 404, 474: 404, 404, 484: 404, 404, 404, 492: 404, 553: 404, 632: 404, 634: 404, 404}, - {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, 1115: 4223}, + {404, 404, 404, 404, 404, 404, 404, 404, 13: 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 458: 404, 404, 404, 464: 404, 404, 404, 404, 474: 404, 404, 484: 404, 404, 404, 491: 404, 553: 404, 632: 404, 634: 404, 404}, + {464: 4222, 493: 2639, 722: 2638, 730: 4221}, + {407, 407, 407, 407, 407, 407, 407, 407, 13: 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 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, 491: 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, 491: 406, 553: 406, 632: 406, 634: 406, 406}, + {464: 4201, 493: 2639, 722: 2638, 730: 4202, 1116: 4224}, // 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}, - {410, 410, 410, 410, 410, 410, 410, 410, 13: 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 458: 410, 410, 410, 464: 410, 410, 410, 410, 474: 410, 410, 484: 410, 410, 410, 492: 410, 553: 410, 632: 410, 634: 410, 410}, - {493: 2638, 722: 2637, 730: 4227}, - {411, 411, 411, 411, 411, 411, 411, 411, 13: 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 458: 411, 411, 411, 464: 411, 411, 411, 411, 474: 411, 411, 484: 411, 411, 411, 492: 411, 553: 411, 632: 411, 634: 411, 411}, + {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, 491: 408, 553: 408, 632: 408, 634: 408, 408}, + {493: 2639, 722: 2638, 730: 4226}, + {410, 410, 410, 410, 410, 410, 410, 410, 13: 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 458: 410, 410, 410, 464: 410, 410, 410, 410, 474: 410, 410, 484: 410, 410, 410, 491: 410, 553: 410, 632: 410, 634: 410, 410}, + {493: 2639, 722: 2638, 730: 4228}, + {411, 411, 411, 411, 411, 411, 411, 411, 13: 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 458: 411, 411, 411, 464: 411, 411, 411, 411, 474: 411, 411, 484: 411, 411, 411, 491: 411, 553: 411, 632: 411, 634: 411, 411}, // 1780 - {461: 4229}, - {412, 412, 412, 412, 412, 412, 412, 412, 13: 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 458: 412, 412, 412, 464: 412, 412, 412, 412, 474: 412, 412, 484: 412, 412, 412, 492: 412, 553: 412, 632: 412, 634: 412, 412}, - {461: 4231}, - {413, 413, 413, 413, 413, 413, 413, 413, 13: 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 458: 413, 413, 413, 464: 413, 413, 413, 413, 474: 413, 413, 484: 413, 413, 413, 492: 413, 553: 413, 632: 413, 634: 413, 413}, - {493: 2638, 722: 2637, 730: 4233}, + {461: 4230}, + {412, 412, 412, 412, 412, 412, 412, 412, 13: 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 458: 412, 412, 412, 464: 412, 412, 412, 412, 474: 412, 412, 484: 412, 412, 412, 491: 412, 553: 412, 632: 412, 634: 412, 412}, + {461: 4232}, + {413, 413, 413, 413, 413, 413, 413, 413, 13: 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 458: 413, 413, 413, 464: 413, 413, 413, 413, 474: 413, 413, 484: 413, 413, 413, 491: 413, 553: 413, 632: 413, 634: 413, 413}, + {493: 2639, 722: 2638, 730: 4234}, // 1785 - {414, 414, 414, 414, 414, 414, 414, 414, 13: 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 458: 414, 414, 414, 464: 414, 414, 414, 414, 474: 414, 414, 484: 414, 414, 414, 492: 414, 553: 414, 632: 414, 634: 414, 414}, - {493: 2638, 722: 2637, 730: 4235}, - {415, 415, 415, 415, 415, 415, 415, 415, 13: 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 458: 415, 415, 415, 464: 415, 415, 415, 415, 474: 415, 415, 484: 415, 415, 415, 492: 415, 553: 415, 632: 415, 634: 415, 415}, - {461: 4237}, - {416, 416, 416, 416, 416, 416, 416, 416, 13: 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 458: 416, 416, 416, 464: 416, 416, 416, 416, 474: 416, 416, 484: 416, 416, 416, 492: 416, 553: 416, 632: 416, 634: 416, 416}, + {414, 414, 414, 414, 414, 414, 414, 414, 13: 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 458: 414, 414, 414, 464: 414, 414, 414, 414, 474: 414, 414, 484: 414, 414, 414, 491: 414, 553: 414, 632: 414, 634: 414, 414}, + {493: 2639, 722: 2638, 730: 4236}, + {415, 415, 415, 415, 415, 415, 415, 415, 13: 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 458: 415, 415, 415, 464: 415, 415, 415, 415, 474: 415, 415, 484: 415, 415, 415, 491: 415, 553: 415, 632: 415, 634: 415, 415}, + {461: 4238}, + {416, 416, 416, 416, 416, 416, 416, 416, 13: 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 458: 416, 416, 416, 464: 416, 416, 416, 416, 474: 416, 416, 484: 416, 416, 416, 491: 416, 553: 416, 632: 416, 634: 416, 416}, // 1790 - {493: 2638, 722: 2637, 730: 4239}, - {417, 417, 417, 417, 417, 417, 417, 417, 13: 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 458: 417, 417, 417, 464: 417, 417, 417, 417, 474: 417, 417, 484: 417, 417, 417, 492: 417, 553: 417, 632: 417, 634: 417, 417}, - {493: 2638, 722: 2637, 730: 4241}, - {419, 419, 419, 419, 419, 419, 419, 419, 13: 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 458: 419, 419, 419, 464: 419, 419, 419, 419, 474: 419, 419, 484: 419, 419, 419, 492: 419, 553: 419, 632: 419, 634: 419, 419}, - {482: 4171, 493: 1987, 723: 4246}, + {493: 2639, 722: 2638, 730: 4240}, + {417, 417, 417, 417, 417, 417, 417, 417, 13: 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 458: 417, 417, 417, 464: 417, 417, 417, 417, 474: 417, 417, 484: 417, 417, 417, 491: 417, 553: 417, 632: 417, 634: 417, 417}, + {493: 2639, 722: 2638, 730: 4242}, + {419, 419, 419, 419, 419, 419, 419, 419, 13: 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 458: 419, 419, 419, 464: 419, 419, 419, 419, 474: 419, 419, 484: 419, 419, 419, 491: 419, 553: 419, 632: 419, 634: 419, 419}, + {482: 4172, 493: 1988, 723: 4247}, // 1795 - {482: 4171, 493: 1987, 723: 4244}, - {493: 2638, 722: 2637, 730: 4245}, - {418, 418, 418, 418, 418, 418, 418, 418, 13: 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 458: 418, 418, 418, 464: 418, 418, 418, 418, 474: 418, 418, 484: 418, 418, 418, 492: 418, 553: 418, 632: 418, 634: 418, 418}, - {493: 2638, 722: 2637, 730: 4247}, - {420, 420, 420, 420, 420, 420, 420, 420, 13: 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 458: 420, 420, 420, 464: 420, 420, 420, 420, 474: 420, 420, 484: 420, 420, 420, 492: 420, 553: 420, 632: 420, 634: 420, 420}, + {482: 4172, 493: 1988, 723: 4245}, + {493: 2639, 722: 2638, 730: 4246}, + {418, 418, 418, 418, 418, 418, 418, 418, 13: 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 458: 418, 418, 418, 464: 418, 418, 418, 418, 474: 418, 418, 484: 418, 418, 418, 491: 418, 553: 418, 632: 418, 634: 418, 418}, + {493: 2639, 722: 2638, 730: 4248}, + {420, 420, 420, 420, 420, 420, 420, 420, 13: 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 458: 420, 420, 420, 464: 420, 420, 420, 420, 474: 420, 420, 484: 420, 420, 420, 491: 420, 553: 420, 632: 420, 634: 420, 420}, // 1800 - {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: 4252}, - {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, 793: 4253}, + {2: 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, 461: 1988, 482: 4172, 529: 1988, 723: 4253}, + {2: 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, 461: 1988, 482: 4172, 529: 1988, 723: 4251}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 529: 3376, 648: 3378, 2665, 2666, 2664, 725: 3375, 853: 4252}, + {421, 421, 421, 421, 421, 421, 421, 421, 13: 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 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, 491: 421, 553: 421, 632: 421, 634: 421, 421}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 529: 3646, 648: 3378, 2665, 2666, 2664, 725: 3645, 794: 4254}, // 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}, - {2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 13: 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 2061, 458: 2061, 2061, 2061, 464: 2061, 2061, 2061, 2061, 474: 2061, 2061, 484: 2061, 2061, 2061, 492: 2061, 553: 2061, 632: 2061, 634: 2061, 2061}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4257, 2664, 2665, 2663}, - {2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 13: 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 458: 2062, 2062, 2062, 464: 2062, 2062, 2062, 2062, 474: 2062, 2062, 484: 2062, 2062, 2062, 492: 2062, 553: 2062, 632: 2062, 634: 2062, 2062}, + {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, 491: 422, 553: 422, 632: 422, 634: 422, 422}, + {493: 2639, 722: 2638, 730: 4256}, + {2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 13: 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 2062, 458: 2062, 2062, 2062, 464: 2062, 2062, 2062, 2062, 474: 2062, 2062, 484: 2062, 2062, 2062, 491: 2062, 553: 2062, 632: 2062, 634: 2062, 2062}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4258, 2665, 2666, 2664}, + {2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 13: 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 458: 2063, 2063, 2063, 464: 2063, 2063, 2063, 2063, 474: 2063, 2063, 484: 2063, 2063, 2063, 491: 2063, 553: 2063, 632: 2063, 634: 2063, 2063}, // 1810 - {493: 2638, 722: 2637, 730: 4259}, - {2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 13: 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 2063, 458: 2063, 2063, 2063, 464: 2063, 2063, 2063, 2063, 474: 2063, 2063, 484: 2063, 2063, 2063, 492: 2063, 553: 2063, 632: 2063, 634: 2063, 2063}, - {493: 2638, 722: 2637, 730: 4261}, - {2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 13: 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 458: 2064, 2064, 2064, 464: 2064, 2064, 2064, 2064, 474: 2064, 2064, 484: 2064, 2064, 2064, 492: 2064, 553: 2064, 632: 2064, 634: 2064, 2064}, - {461: 1987, 482: 4171, 723: 4263}, + {493: 2639, 722: 2638, 730: 4260}, + {2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 13: 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 2064, 458: 2064, 2064, 2064, 464: 2064, 2064, 2064, 2064, 474: 2064, 2064, 484: 2064, 2064, 2064, 491: 2064, 553: 2064, 632: 2064, 634: 2064, 2064}, + {493: 2639, 722: 2638, 730: 4262}, + {2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 13: 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 458: 2065, 2065, 2065, 464: 2065, 2065, 2065, 2065, 474: 2065, 2065, 484: 2065, 2065, 2065, 491: 2065, 553: 2065, 632: 2065, 634: 2065, 2065}, + {461: 1988, 482: 4172, 723: 4264}, // 1815 - {461: 4264}, - {2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 13: 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 2065, 458: 2065, 2065, 2065, 464: 2065, 2065, 2065, 2065, 474: 2065, 2065, 484: 2065, 2065, 2065, 492: 2065, 553: 2065, 632: 2065, 634: 2065, 2065}, - {461: 1987, 482: 4171, 723: 4266}, - {461: 4267}, - {2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 13: 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 458: 2066, 2066, 2066, 464: 2066, 2066, 2066, 2066, 474: 2066, 2066, 484: 2066, 2066, 2066, 492: 2066, 553: 2066, 632: 2066, 634: 2066, 2066}, + {461: 4265}, + {2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 13: 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 2066, 458: 2066, 2066, 2066, 464: 2066, 2066, 2066, 2066, 474: 2066, 2066, 484: 2066, 2066, 2066, 491: 2066, 553: 2066, 632: 2066, 634: 2066, 2066}, + {461: 1988, 482: 4172, 723: 4267}, + {461: 4268}, + {2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 13: 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 458: 2067, 2067, 2067, 464: 2067, 2067, 2067, 2067, 474: 2067, 2067, 484: 2067, 2067, 2067, 491: 2067, 553: 2067, 632: 2067, 634: 2067, 2067}, // 1820 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4269}, - {2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 13: 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 2067, 458: 2067, 2067, 2067, 464: 2067, 2067, 2067, 2067, 474: 2067, 2067, 484: 2067, 2067, 2067, 492: 2067, 553: 2067, 632: 2067, 634: 2067, 2067}, - {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, 723: 4273}, - {395, 395, 395, 395, 395, 395, 395, 395, 13: 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 458: 395, 395, 395, 464: 395, 395, 395, 395, 474: 395, 395, 484: 395, 395, 395, 492: 395, 553: 395, 632: 395, 634: 395, 395}, - {394, 394, 394, 394, 394, 394, 394, 394, 13: 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 458: 394, 394, 394, 464: 394, 394, 394, 394, 474: 394, 394, 484: 394, 394, 394, 492: 394, 553: 394, 632: 394, 634: 394, 394}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 648: 3378, 2665, 2666, 2664, 725: 4270}, + {2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 13: 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 458: 2068, 2068, 2068, 464: 2068, 2068, 2068, 2068, 474: 2068, 2068, 484: 2068, 2068, 2068, 491: 2068, 553: 2068, 632: 2068, 634: 2068, 2068}, + {2: 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, 461: 1988, 482: 4172, 723: 4274}, + {395, 395, 395, 395, 395, 395, 395, 395, 13: 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 458: 395, 395, 395, 464: 395, 395, 395, 395, 474: 395, 395, 484: 395, 395, 395, 491: 395, 553: 395, 632: 395, 634: 395, 395}, + {394, 394, 394, 394, 394, 394, 394, 394, 13: 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 458: 394, 394, 394, 464: 394, 394, 394, 394, 474: 394, 394, 484: 394, 394, 394, 491: 394, 553: 394, 632: 394, 634: 394, 394}, // 1825 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4274}, - {2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 13: 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 2068, 458: 2068, 2068, 2068, 464: 2068, 2068, 2068, 2068, 474: 2068, 2068, 484: 2068, 2068, 2068, 492: 2068, 553: 2068, 632: 2068, 634: 2068, 2068}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4276}, - {2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 13: 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 458: 2069, 2069, 2069, 464: 2069, 2069, 2069, 2069, 474: 2069, 2069, 484: 2069, 2069, 2069, 492: 2069, 553: 2069, 632: 2069, 634: 2069, 2069}, - {461: 4278}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 648: 3378, 2665, 2666, 2664, 725: 4275}, + {2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 13: 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 2069, 458: 2069, 2069, 2069, 464: 2069, 2069, 2069, 2069, 474: 2069, 2069, 484: 2069, 2069, 2069, 491: 2069, 553: 2069, 632: 2069, 634: 2069, 2069}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 648: 3378, 2665, 2666, 2664, 725: 4277}, + {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, 491: 2070, 553: 2070, 632: 2070, 634: 2070, 2070}, + {461: 4279}, // 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, 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}, + {2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 13: 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 2071, 458: 2071, 2071, 2071, 464: 2071, 2071, 2071, 2071, 474: 2071, 2071, 484: 2071, 2071, 2071, 491: 2071, 553: 2071, 632: 2071, 634: 2071, 2071}, + {4: 4132, 4134, 389, 13: 2107, 4151, 4078, 4090, 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 4149, 4169, 4153, 4140, 4133, 4136, 4135, 4138, 4139, 4141, 4148, 389, 4159, 4160, 4146, 4147, 4152, 4154, 4166, 4165, 4171, 4167, 4164, 4157, 4162, 4163, 4156, 4158, 4161, 4150, 80: 4103, 83: 4124, 4125, 92: 4126, 132: 4106, 190: 4091, 4110, 194: 4111, 207: 4105, 214: 4121, 225: 4100, 235: 4107, 239: 4102, 254: 4112, 262: 4108, 269: 4122, 4123, 276: 4092, 460: 4120, 464: 4131, 466: 4168, 2107, 483: 4127, 485: 4119, 2107, 490: 4109, 498: 4094, 571: 4099, 4095, 634: 2107, 4137, 653: 4114, 656: 4101, 658: 4128, 666: 4113, 673: 4115, 676: 4096, 691: 4104, 758: 4088, 762: 4089, 766: 4142, 779: 4144, 798: 4143, 821: 4145, 825: 4155, 829: 4170, 850: 4118, 865: 4116, 904: 4093, 910: 4097, 971: 4281, 1115: 4098, 1142: 4117}, + {2327, 2327, 2327, 2327, 7: 2327, 475: 2327}, + {2341, 2341, 2341, 2341, 7: 2341, 475: 2341}, {2340, 2340, 2340, 2340, 7: 2340, 475: 2340}, - {2339, 2339, 2339, 2339, 7: 2339, 475: 2339}, // 1835 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4284, 648: 4285, 2664, 2665, 2663}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 464: 4285, 648: 4286, 2665, 2666, 2664}, + {2343, 2343, 2343, 2343, 7: 2343, 92: 2343, 475: 2343}, {2342, 2342, 2342, 2342, 7: 2342, 92: 2342, 475: 2342}, - {2341, 2341, 2341, 2341, 7: 2341, 92: 2341, 475: 2341}, - {161: 4291, 228: 4288, 246: 4289, 4290, 464: 4287}, - {2347, 2347, 2347, 2347, 7: 2347, 475: 2347, 483: 2347}, + {163: 4292, 229: 4289, 247: 4290, 4291, 464: 4288}, + {2348, 2348, 2348, 2348, 7: 2348, 475: 2348, 483: 2348}, // 1840 + {2347, 2347, 2347, 2347, 7: 2347, 475: 2347, 483: 2347}, {2346, 2346, 2346, 2346, 7: 2346, 475: 2346, 483: 2346}, {2345, 2345, 2345, 2345, 7: 2345, 475: 2345, 483: 2345}, {2344, 2344, 2344, 2344, 7: 2344, 475: 2344, 483: 2344}, - {2343, 2343, 2343, 2343, 7: 2343, 475: 2343, 483: 2343}, - {2365, 2365, 2365, 2365, 7: 2365, 475: 2365}, - // 1845 {2366, 2366, 2366, 2366, 7: 2366, 475: 2366}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4307, 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, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4306}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4305}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4304}, + // 1845 + {2367, 2367, 2367, 2367, 7: 2367, 475: 2367}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4308, 2665, 2666, 2664}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 4307}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 4306}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 4305}, // 1850 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4301, 2664, 2665, 2663}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4302, 2665, 2666, 2664}, + {2: 2339, 2339, 2339, 2339, 2339, 8: 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 58: 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 2339, 459: 2339, 468: 2339, 479: 2339, 552: 2339}, {2: 2338, 2338, 2338, 2338, 2338, 8: 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 58: 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 2338, 459: 2338, 468: 2338, 479: 2338, 552: 2338}, - {2: 2337, 2337, 2337, 2337, 2337, 8: 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 58: 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 459: 2337, 468: 2337, 479: 2337, 552: 2337}, - {637: 4302}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4303, 2664, 2665, 2663}, + {637: 4303}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4304, 2665, 2666, 2664}, // 1855 - {2371, 2371, 2371, 2371, 7: 2371, 475: 2371}, {2372, 2372, 2372, 2372, 7: 2372, 475: 2372}, {2373, 2373, 2373, 2373, 7: 2373, 475: 2373}, {2374, 2374, 2374, 2374, 7: 2374, 475: 2374}, - {637: 4308}, - // 1860 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4309, 2664, 2665, 2663}, {2375, 2375, 2375, 2375, 7: 2375, 475: 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, 648: 4024, 2664, 2665, 2663, 729: 4325}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4320, 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, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4316, 2664, 2665, 2663}, + {637: 4309}, + // 1860 + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4310, 2665, 2666, 2664}, + {2376, 2376, 2376, 2376, 7: 2376, 475: 2376}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4326}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4321, 2665, 2666, 2664}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4317, 2665, 2666, 2664}, // 1865 - {2: 2333, 2333, 2333, 2333, 2333, 8: 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 58: 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 2333, 459: 2333, 552: 2333}, + {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, 552: 2334}, {2: 431, 431, 431, 431, 431, 8: 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 58: 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431}, {2: 430, 430, 430, 430, 430, 8: 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 58: 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430}, - {95: 4319, 97: 4318, 839: 4317}, - {2360, 2360, 2360, 2360, 7: 2360, 475: 2360}, + {95: 4320, 97: 4319, 840: 4318}, + {2361, 2361, 2361, 2361, 7: 2361, 475: 2361}, // 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, 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}, + {146: 4323, 462: 3971, 3970, 796: 4324, 918: 4322}, + {2363, 2363, 2363, 2363, 7: 2363, 475: 2363}, + {2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 57: 2231, 458: 2231, 462: 2231, 2231, 2231, 2231, 467: 2231, 475: 2231, 478: 2231, 563: 2231, 571: 2231, 573: 2231, 629: 2231, 2231, 2231, 633: 2231}, // 1875 - {146: 4324}, - {2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 57: 2229, 458: 2229, 462: 2229, 2229, 2229, 2229, 467: 2229, 475: 2229, 478: 2229, 563: 2229, 571: 2229, 573: 2229, 629: 2229, 2229, 2229, 633: 2229}, - {498: 4326, 656: 4327}, + {146: 4325}, + {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}, + {498: 4327, 656: 4328}, + {464: 4330}, {464: 4329}, - {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, 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}, + {2377, 2377, 2377, 2377, 7: 2377, 475: 2377}, + {459: 4332, 461: 3128, 471: 4335, 4334, 478: 3119, 493: 3123, 557: 3118, 3120, 3122, 3121, 562: 3126, 566: 3127, 579: 3125, 699: 4333, 3124, 1111: 4331}, + {2379, 2379, 2379, 2379, 7: 2379, 475: 2379}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 4338}, + {2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 57: 2153, 458: 2153, 462: 2153, 2153, 2153, 2153, 467: 2153, 475: 2153, 478: 2153, 563: 2153, 571: 2153, 573: 2153, 629: 2153, 2153, 2153, 633: 2153}, // 1885 - {493: 3309, 559: 3311, 3310, 805: 4336}, - {493: 3309, 559: 3311, 3310, 805: 4335}, - {2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 2150, 57: 2150, 458: 2150, 462: 2150, 2150, 2150, 2150, 467: 2150, 475: 2150, 478: 2150, 563: 2150, 571: 2150, 573: 2150, 629: 2150, 2150, 2150, 633: 2150}, + {493: 3310, 559: 3312, 3311, 806: 4337}, + {493: 3310, 559: 3312, 3311, 806: 4336}, {2151, 2151, 2151, 2151, 2151, 2151, 2151, 2151, 2151, 2151, 2151, 2151, 2151, 57: 2151, 458: 2151, 462: 2151, 2151, 2151, 2151, 467: 2151, 475: 2151, 478: 2151, 563: 2151, 571: 2151, 573: 2151, 629: 2151, 2151, 2151, 633: 2151}, - {57: 4338, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, + {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}, + {57: 4339, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, // 1890 - {2377, 2377, 2377, 2377, 7: 2377, 475: 2377}, - {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: 4340}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4343}, - {569: 4342}, + {2378, 2378, 2378, 2378, 7: 2378, 475: 2378}, + {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: 4342, 765: 4341}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4344}, + {569: 4343}, {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, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4346, 832: 4345}, + {2333, 2333, 2333, 2333, 7: 2333, 4603, 4604, 475: 2333, 913: 4602}, + {10: 4348, 105: 4396, 109: 4397, 170: 4407, 4406, 4372, 174: 4387, 187: 4409, 213: 4408, 219: 4369, 296: 4376, 4368, 317: 4385, 343: 4392, 4391, 347: 4395, 380: 4403, 486: 4390, 498: 4386, 529: 4381, 634: 4389, 664: 4394, 4393, 667: 4370, 4375, 4373, 4366, 4360, 4374, 675: 4382, 677: 4367, 4399, 4361, 4362, 4363, 4364, 4365, 4388, 4401, 4405, 4400, 4359, 4404, 4371, 692: 4358, 4398, 4357, 4402, 886: 4377, 1134: 4379, 1156: 4356, 4383, 4353, 1176: 4351, 1190: 4354, 1192: 4355, 1211: 4352, 1228: 4378, 4349, 4380, 1285: 4350, 1297: 4384, 1300: 4347, 1325: 4410}, + {2194, 2194, 2194, 2194, 4490, 4496, 4484, 2194, 2194, 2194, 4488, 4497, 4495, 57: 2194, 458: 4489, 462: 3971, 3970, 4487, 2201, 467: 4494, 475: 2194, 478: 4483, 563: 2235, 571: 2324, 573: 4481, 629: 4486, 4479, 4501, 633: 4498, 796: 4482, 819: 4491, 895: 4493, 912: 4499, 921: 4492, 937: 4485, 985: 4500, 4601}, + {2194, 2194, 2194, 2194, 4490, 4496, 4484, 2194, 2194, 2194, 4488, 4497, 4495, 57: 2194, 458: 4489, 462: 3971, 3970, 4487, 2201, 467: 4494, 475: 2194, 478: 4483, 563: 2235, 571: 2324, 573: 4481, 629: 4486, 4479, 4501, 633: 4498, 796: 4482, 819: 4491, 895: 4493, 912: 4499, 921: 4492, 937: 4485, 985: 4500, 4480}, // 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}, {361, 361, 361, 361, 361, 361, 361, 361, 361, 361, 361, 361, 361, 57: 361, 458: 361, 462: 361, 361, 361, 361, 467: 361, 475: 361, 478: 361, 563: 361, 571: 361, 573: 361, 629: 361, 361, 361, 633: 361}, - {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}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 57: 278, 60: 278, 458: 278, 3711, 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: 3712, 771: 4477}, + {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, 857: 4476}, // 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, 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}, + {271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 57: 271, 60: 271, 458: 271, 3698, 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: 3699, 889: 4474, 894: 3700}, + {271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 57: 271, 60: 271, 458: 271, 3698, 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: 3699, 889: 4472, 894: 3700}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 57: 278, 458: 278, 3711, 462: 278, 278, 278, 278, 467: 278, 475: 278, 478: 278, 563: 278, 571: 278, 573: 278, 629: 278, 278, 278, 633: 278, 747: 3712, 771: 4471}, {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}, // 1910 @@ -8314,27 +8315,27 @@ var ( {336, 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, 57: 336, 60: 336, 458: 336, 336, 462: 336, 336, 336, 336, 467: 336, 475: 336, 478: 336, 563: 336, 571: 336, 573: 336, 629: 336, 336, 336, 633: 336, 726: 336, 728: 336}, // 1925 {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}, + {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: 4470}, {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, 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}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3721, 57: 265, 458: 265, 3711, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3722, 529: 3718, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3720, 747: 4467, 759: 3719, 786: 4468}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3721, 57: 265, 458: 265, 3711, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3722, 529: 3718, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3720, 747: 4464, 759: 3719, 786: 4465}, // 1930 - {459: 3710, 747: 4461}, - {459: 3710, 747: 4459}, - {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: 4458}, - {459: 3710, 747: 4457}, + {459: 3711, 747: 4462}, + {459: 3711, 747: 4460}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 57: 278, 458: 278, 3711, 462: 278, 278, 278, 278, 467: 278, 475: 278, 478: 278, 563: 278, 571: 278, 573: 278, 629: 278, 278, 278, 633: 278, 747: 3712, 771: 4459}, + {459: 3711, 747: 4458}, {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, 784: 4437, 863: 4456}, - {459: 4452}, - {459: 4442}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3721, 57: 265, 101: 4439, 4441, 104: 4440, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3722, 529: 3718, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3720, 759: 3719, 786: 4438, 866: 4457}, + {459: 4453}, + {459: 4443}, {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, 784: 4437, 863: 4433, 1134: 4432}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3721, 57: 265, 101: 4439, 4441, 104: 4440, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 4436, 529: 3718, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 4435, 664: 4394, 4393, 675: 4437, 759: 3719, 786: 4438, 866: 4434, 1134: 4433}, // 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}, - {314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 57: 314, 458: 314, 314, 462: 314, 314, 314, 314, 467: 314, 475: 314, 478: 314, 486: 314, 529: 314, 563: 314, 571: 314, 573: 314, 629: 314, 314, 314, 633: 314, 314, 664: 4428, 4427, 829: 4429}, - {486: 4422, 634: 4421, 664: 4424, 4423}, + {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, 830: 4432}, + {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, 830: 4431}, + {314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 57: 314, 458: 314, 314, 462: 314, 314, 314, 314, 467: 314, 475: 314, 478: 314, 486: 314, 529: 314, 563: 314, 571: 314, 573: 314, 629: 314, 314, 314, 633: 314, 314, 664: 4429, 4428, 830: 4430}, + {486: 4423, 634: 4422, 664: 4425, 4424}, {309, 309, 309, 309, 309, 309, 309, 309, 309, 309, 309, 309, 309, 309, 57: 309, 101: 309, 309, 104: 309, 458: 309, 309, 462: 309, 309, 309, 309, 467: 309, 475: 309, 478: 309, 486: 309, 529: 309, 563: 309, 571: 309, 573: 309, 629: 309, 309, 309, 633: 309, 309}, // 1945 {308, 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, 57: 308, 101: 308, 308, 104: 308, 458: 308, 308, 462: 308, 308, 308, 308, 467: 308, 475: 308, 478: 308, 486: 308, 529: 308, 563: 308, 571: 308, 573: 308, 629: 308, 308, 308, 633: 308, 308}, @@ -8343,22 +8344,22 @@ var ( {298, 298, 298, 298, 298, 298, 298, 298, 298, 298, 298, 298, 298, 57: 298, 60: 298, 458: 298, 298, 462: 298, 298, 298, 298, 467: 298, 475: 298, 478: 298, 563: 298, 571: 298, 573: 298, 629: 298, 298, 298, 633: 298, 726: 298, 728: 298}, {297, 297, 297, 297, 297, 297, 297, 297, 297, 297, 297, 297, 297, 57: 297, 458: 297, 462: 297, 297, 297, 297, 467: 297, 475: 297, 478: 297, 563: 297, 571: 297, 573: 297, 629: 297, 297, 297, 633: 297}, // 1950 - {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: 4420}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 57: 278, 458: 278, 3711, 462: 278, 278, 278, 278, 467: 278, 475: 278, 478: 278, 563: 278, 571: 278, 573: 278, 629: 278, 278, 278, 633: 278, 747: 3712, 771: 4421}, {295, 295, 295, 295, 295, 295, 295, 295, 295, 295, 295, 295, 295, 57: 295, 458: 295, 462: 295, 295, 295, 295, 467: 295, 475: 295, 478: 295, 563: 295, 571: 295, 573: 295, 629: 295, 295, 295, 633: 295}, {294, 294, 294, 294, 294, 294, 294, 294, 294, 294, 294, 294, 294, 57: 294, 458: 294, 462: 294, 294, 294, 294, 467: 294, 475: 294, 478: 294, 563: 294, 571: 294, 573: 294, 629: 294, 294, 294, 633: 294}, {292, 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, 57: 292, 101: 292, 292, 104: 292, 458: 292, 462: 292, 292, 292, 292, 467: 292, 475: 292, 478: 292, 486: 292, 529: 292, 563: 292, 571: 292, 573: 292, 629: 292, 292, 292, 633: 292, 292}, - {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 57: 278, 101: 278, 278, 104: 278, 458: 278, 3710, 462: 278, 278, 278, 278, 467: 278, 475: 278, 478: 278, 486: 278, 529: 278, 563: 278, 571: 278, 573: 278, 629: 278, 278, 278, 633: 278, 278, 747: 3711, 770: 4419}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 57: 278, 101: 278, 278, 104: 278, 458: 278, 3711, 462: 278, 278, 278, 278, 467: 278, 475: 278, 478: 278, 486: 278, 529: 278, 563: 278, 571: 278, 573: 278, 629: 278, 278, 278, 633: 278, 278, 747: 3712, 771: 4420}, // 1955 {290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 57: 290, 101: 290, 290, 104: 290, 458: 290, 462: 290, 290, 290, 290, 467: 290, 475: 290, 478: 290, 486: 290, 529: 290, 563: 290, 571: 290, 573: 290, 629: 290, 290, 290, 633: 290, 290}, {289, 289, 289, 289, 289, 289, 289, 289, 289, 289, 289, 289, 289, 289, 57: 289, 101: 289, 289, 104: 289, 458: 289, 462: 289, 289, 289, 289, 467: 289, 475: 289, 478: 289, 486: 289, 529: 289, 563: 289, 571: 289, 573: 289, 629: 289, 289, 289, 633: 289, 289}, {284, 284, 284, 284, 284, 284, 284, 284, 284, 284, 284, 284, 284, 57: 284, 458: 284, 462: 284, 284, 284, 284, 467: 284, 475: 284, 478: 284, 563: 284, 571: 284, 573: 284, 629: 284, 284, 284, 633: 284}, - {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: 4418}, - {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: 4417}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 57: 278, 458: 278, 3711, 462: 278, 278, 278, 278, 467: 278, 475: 278, 478: 278, 563: 278, 571: 278, 573: 278, 629: 278, 278, 278, 633: 278, 747: 3712, 771: 4419}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 57: 278, 458: 278, 3711, 462: 278, 278, 278, 278, 467: 278, 475: 278, 478: 278, 563: 278, 571: 278, 573: 278, 629: 278, 278, 278, 633: 278, 747: 3712, 771: 4418}, // 1960 - {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: 4416}, - {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: 4410}, - {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: 4411}, - {280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 57: 280, 60: 4413, 458: 280, 462: 280, 280, 280, 280, 467: 280, 475: 280, 478: 280, 563: 280, 571: 280, 573: 280, 629: 280, 280, 280, 633: 280, 726: 4412, 728: 4414, 854: 4415}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 57: 278, 458: 278, 3711, 462: 278, 278, 278, 278, 467: 278, 475: 278, 478: 278, 563: 278, 571: 278, 573: 278, 629: 278, 278, 278, 633: 278, 747: 3712, 771: 4417}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 57: 278, 60: 278, 458: 278, 3711, 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: 3712, 771: 4411}, + {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, 857: 4412}, + {280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 57: 280, 60: 4414, 458: 280, 462: 280, 280, 280, 280, 467: 280, 475: 280, 478: 280, 563: 280, 571: 280, 573: 280, 629: 280, 280, 280, 633: 280, 726: 4413, 728: 4415, 856: 4416}, {276, 276, 276, 276, 276, 276, 276, 276, 276, 276, 276, 276, 276, 57: 276, 60: 276, 458: 276, 462: 276, 276, 276, 276, 467: 276, 475: 276, 478: 276, 563: 276, 571: 276, 573: 276, 629: 276, 276, 276, 633: 276, 726: 276, 728: 276}, // 1965 {275, 275, 275, 275, 275, 275, 275, 275, 275, 275, 275, 275, 275, 57: 275, 60: 275, 458: 275, 462: 275, 275, 275, 275, 467: 275, 475: 275, 478: 275, 563: 275, 571: 275, 573: 275, 629: 275, 275, 275, 633: 275, 726: 275, 728: 275}, @@ -8370,8 +8371,8 @@ var ( {283, 283, 283, 283, 283, 283, 283, 283, 283, 283, 283, 283, 283, 57: 283, 458: 283, 462: 283, 283, 283, 283, 467: 283, 475: 283, 478: 283, 563: 283, 571: 283, 573: 283, 629: 283, 283, 283, 633: 283}, {291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 57: 291, 101: 291, 291, 104: 291, 458: 291, 462: 291, 291, 291, 291, 467: 291, 475: 291, 478: 291, 486: 291, 529: 291, 563: 291, 571: 291, 573: 291, 629: 291, 291, 291, 633: 291, 291}, {296, 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, 57: 296, 458: 296, 462: 296, 296, 296, 296, 467: 296, 475: 296, 478: 296, 563: 296, 571: 296, 573: 296, 629: 296, 296, 296, 633: 296}, - {313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 57: 313, 458: 313, 313, 462: 313, 313, 313, 313, 467: 313, 475: 313, 478: 313, 486: 313, 529: 313, 563: 313, 571: 313, 573: 313, 629: 313, 313, 313, 633: 313, 313, 829: 4426}, - {312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 57: 312, 458: 312, 312, 462: 312, 312, 312, 312, 467: 312, 475: 312, 478: 312, 486: 312, 529: 312, 563: 312, 571: 312, 573: 312, 629: 312, 312, 312, 633: 312, 312, 829: 4425}, + {313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 57: 313, 458: 313, 313, 462: 313, 313, 313, 313, 467: 313, 475: 313, 478: 313, 486: 313, 529: 313, 563: 313, 571: 313, 573: 313, 629: 313, 313, 313, 633: 313, 313, 830: 4427}, + {312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 57: 312, 458: 312, 312, 462: 312, 312, 312, 312, 467: 312, 475: 312, 478: 312, 486: 312, 529: 312, 563: 312, 571: 312, 573: 312, 629: 312, 312, 312, 633: 312, 312, 830: 4426}, // 1975 {459: 307}, {459: 306}, @@ -8383,11 +8384,11 @@ 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, 784: 4437, 863: 4441}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3721, 57: 265, 101: 4439, 4441, 104: 4440, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3722, 529: 3718, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3720, 759: 3719, 786: 4438, 866: 4442}, // 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}, - {498: 3722, 829: 4430}, + {498: 3724, 830: 4432}, + {498: 3723, 830: 4431}, {293, 293, 293, 293, 293, 293, 293, 293, 293, 293, 293, 293, 293, 57: 293, 458: 293, 462: 293, 293, 293, 293, 467: 293, 475: 293, 478: 293, 563: 293, 571: 293, 573: 293, 629: 293, 293, 293, 633: 293}, {288, 288, 288, 288, 288, 288, 288, 288, 288, 288, 288, 288, 288, 57: 288, 458: 288, 462: 288, 288, 288, 288, 467: 288, 475: 288, 478: 288, 563: 288, 571: 288, 573: 288, 629: 288, 288, 288, 633: 288}, // 1990 @@ -8395,744 +8396,744 @@ 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, 951: 4447, 1127: 4443}, + {461: 4445, 562: 4446, 566: 4447, 951: 4448, 1128: 4444}, // 1995 - {7: 4449, 57: 4448}, + {7: 4450, 57: 4449}, {7: 253, 57: 253}, {7: 252, 57: 252}, {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, 784: 4437, 863: 4451}, - {461: 4444, 562: 4445, 566: 4446, 951: 4450}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3721, 57: 265, 101: 4439, 4441, 104: 4440, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3722, 529: 3718, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3720, 759: 3719, 786: 4438, 866: 4452}, + {461: 4445, 562: 4446, 566: 4447, 951: 4451}, {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, 951: 4447, 1127: 4453}, + {461: 4445, 562: 4446, 566: 4447, 951: 4448, 1128: 4454}, // 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, 784: 4437, 863: 4455}, + {7: 4450, 57: 4455}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3721, 57: 265, 101: 4439, 4441, 104: 4440, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3722, 529: 3718, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3720, 759: 3719, 786: 4438, 866: 4456}, {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, 784: 4460}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3721, 57: 265, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3722, 529: 3718, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3720, 759: 3719, 786: 4461}, {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, 784: 4462}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3721, 57: 265, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3722, 529: 3718, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3720, 759: 3719, 786: 4463}, {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, 784: 4465}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3721, 57: 265, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3722, 529: 3718, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3720, 759: 3719, 786: 4466}, {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, 784: 4468}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 3721, 57: 265, 458: 265, 462: 265, 265, 265, 265, 467: 265, 475: 265, 478: 265, 486: 3722, 529: 3718, 563: 265, 571: 265, 573: 265, 629: 265, 265, 265, 633: 265, 3720, 759: 3719, 786: 4469}, {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}, {333, 333, 333, 333, 333, 333, 333, 333, 333, 333, 333, 333, 333, 57: 333, 60: 333, 458: 333, 333, 462: 333, 333, 333, 333, 467: 333, 475: 333, 478: 333, 563: 333, 571: 333, 573: 333, 629: 333, 333, 333, 633: 333, 726: 333, 728: 333}, {356, 356, 356, 356, 356, 356, 356, 356, 356, 356, 356, 356, 356, 57: 356, 458: 356, 462: 356, 356, 356, 356, 467: 356, 475: 356, 478: 356, 563: 356, 571: 356, 573: 356, 629: 356, 356, 356, 633: 356}, - {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: 4472}, - {357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 57: 357, 60: 4413, 458: 357, 462: 357, 357, 357, 357, 467: 357, 475: 357, 478: 357, 563: 357, 571: 357, 573: 357, 629: 357, 357, 357, 633: 357, 726: 4412, 728: 4414, 854: 4415}, + {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, 857: 4473}, + {357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 57: 357, 60: 4414, 458: 357, 462: 357, 357, 357, 357, 467: 357, 475: 357, 478: 357, 563: 357, 571: 357, 573: 357, 629: 357, 357, 357, 633: 357, 726: 4413, 728: 4415, 856: 4416}, // 2025 - {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: 4474}, - {358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 57: 358, 60: 4413, 458: 358, 462: 358, 358, 358, 358, 467: 358, 475: 358, 478: 358, 563: 358, 571: 358, 573: 358, 629: 358, 358, 358, 633: 358, 726: 4412, 728: 4414, 854: 4415}, - {359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 57: 359, 60: 4413, 458: 359, 462: 359, 359, 359, 359, 467: 359, 475: 359, 478: 359, 563: 359, 571: 359, 573: 359, 629: 359, 359, 359, 633: 359, 726: 4412, 728: 4414, 854: 4415}, - {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}, + {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, 857: 4475}, + {358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 57: 358, 60: 4414, 458: 358, 462: 358, 358, 358, 358, 467: 358, 475: 358, 478: 358, 563: 358, 571: 358, 573: 358, 629: 358, 358, 358, 633: 358, 726: 4413, 728: 4415, 856: 4416}, + {359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 57: 359, 60: 4414, 458: 359, 462: 359, 359, 359, 359, 467: 359, 475: 359, 478: 359, 563: 359, 571: 359, 573: 359, 629: 359, 359, 359, 633: 359, 726: 4413, 728: 4415, 856: 4416}, + {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, 857: 4478}, + {360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 57: 360, 60: 4414, 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: 4413, 728: 4415, 856: 4416}, // 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, 1120: 4598}, - {2256, 2256, 2256, 2256, 7: 2256, 2256, 2256, 57: 2256, 475: 2256}, - {563: 2233}, - {478: 4597}, - {2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 57: 2223, 458: 2223, 462: 2223, 2223, 2223, 2223, 467: 2223, 475: 2223, 478: 2223, 563: 2223, 571: 2223, 573: 2223, 629: 2223, 2223, 2223, 633: 2223}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 563: 2323, 571: 2323, 573: 2323, 629: 2323, 635: 2323, 648: 4600, 2665, 2666, 2664, 662: 2323, 2323, 1121: 4599}, + {2257, 2257, 2257, 2257, 7: 2257, 2257, 2257, 57: 2257, 475: 2257}, + {563: 2234}, + {478: 4598}, + {2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 57: 2224, 458: 2224, 462: 2224, 2224, 2224, 2224, 467: 2224, 475: 2224, 478: 2224, 563: 2224, 571: 2224, 573: 2224, 629: 2224, 2224, 2224, 633: 2224}, // 2035 - {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, 1057: 4574, 4575, 4587, 1110: 4588, 1178: 4586}, - {464: 4584}, + {2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 57: 2223, 458: 2223, 462: 2223, 2223, 2223, 2223, 467: 2223, 475: 2223, 478: 2223, 563: 2223, 571: 2223, 573: 2223, 629: 2223, 2223, 2223, 633: 2223}, + {563: 4594}, + {2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 57: 2220, 458: 2220, 462: 2220, 2220, 2220, 2220, 467: 2220, 475: 2220, 478: 2220, 563: 4593, 571: 2220, 573: 2220, 629: 2220, 2220, 2220, 633: 2220}, + {255: 4591, 345: 4592, 461: 3128, 471: 4335, 4334, 478: 3119, 493: 3123, 557: 3118, 3120, 3122, 3121, 562: 3126, 566: 3127, 575: 4580, 4577, 4578, 4579, 3125, 699: 4333, 3124, 4590, 1058: 4575, 4576, 4588, 1111: 4589, 1178: 4587}, + {464: 4585}, // 2040 - {641: 4572}, - {461: 4571}, - {571: 4562}, - {465: 4555}, - {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}, + {641: 4573}, + {461: 4572}, + {571: 4563}, + {465: 4556}, + {2212, 2212, 2212, 2212, 2212, 2212, 2212, 2212, 2212, 2212, 2212, 2212, 2212, 57: 2212, 458: 2212, 462: 2212, 2212, 2212, 2212, 467: 2212, 475: 2212, 478: 2212, 563: 2212, 571: 2212, 573: 2212, 629: 2212, 2212, 2212, 633: 2212}, // 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, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 529: 3376, 648: 3378, 2665, 2666, 2664, 725: 3375, 853: 4555}, + {172: 4553, 193: 4554, 464: 4552, 1163: 4551}, + {176: 4550, 236: 4549, 464: 4548, 1281: 4547}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 57: 278, 458: 278, 3711, 462: 278, 278, 278, 278, 467: 278, 475: 278, 478: 278, 563: 278, 571: 278, 573: 278, 629: 278, 278, 278, 633: 278, 747: 3712, 771: 4546}, + {293: 4545}, // 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, 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}, + {2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 57: 2196, 458: 2196, 462: 2196, 2196, 2196, 2196, 467: 2196, 475: 2196, 478: 2196, 563: 2196, 571: 2196, 573: 2196, 629: 2196, 2196, 2196, 633: 2196}, + {2193, 2193, 2193, 2193, 4490, 4496, 4484, 2193, 2193, 2193, 4488, 4497, 4495, 57: 2193, 458: 4489, 462: 3971, 3970, 4487, 2201, 467: 4494, 475: 2193, 478: 4483, 563: 2235, 571: 2324, 573: 4481, 629: 4486, 4479, 4501, 633: 4498, 796: 4482, 819: 4491, 895: 4493, 912: 4544, 921: 4492, 937: 4485}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 4502}, + {2141, 2141, 2141, 2141, 2141, 2141, 2141, 2141, 2141, 2141, 2141, 2141, 2141, 57: 2141, 458: 2141, 4504, 462: 2141, 2141, 2141, 2141, 467: 2141, 475: 2141, 478: 2141, 563: 2141, 571: 2141, 573: 2141, 629: 2141, 2141, 2141, 633: 2141, 636: 2141, 1207: 4503}, + {2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 57: 2183, 458: 2183, 462: 2183, 2183, 2183, 2183, 467: 2183, 475: 2183, 478: 2183, 563: 2183, 571: 2183, 573: 2183, 629: 2183, 2183, 2183, 633: 2183, 636: 4519, 1224: 4520, 4521}, // 2055 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4504}, - {7: 4516, 57: 4515}, - {7: 2138, 57: 2138}, - {7: 278, 57: 278, 459: 3710, 515: 278, 278, 747: 3711, 770: 4513}, - {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 4508, 648: 4025, 2665, 2666, 2664, 729: 4507, 813: 4506, 822: 4505}, + {7: 4517, 57: 4516}, + {7: 2139, 57: 2139}, + {7: 278, 57: 278, 459: 3711, 515: 278, 278, 747: 3712, 771: 4514}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 4509}, // 2060 - {57: 4509, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {7: 1261, 57: 1261, 515: 4512, 4511, 930: 4510}, - {7: 2135, 57: 2135}, + {57: 4510, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {7: 1261, 57: 1261, 515: 4513, 4512, 930: 4511}, + {7: 2136, 57: 2136}, {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, 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}, + {7: 1261, 57: 1261, 515: 4513, 4512, 930: 4515}, {7: 2137, 57: 2137}, + {2140, 2140, 2140, 2140, 2140, 2140, 2140, 2140, 2140, 2140, 2140, 2140, 2140, 57: 2140, 458: 2140, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 4508, 648: 4025, 2665, 2666, 2664, 729: 4507, 813: 4518}, + {7: 2138, 57: 2138}, // 2070 - {197: 4540, 352: 4541, 369: 4542}, + {197: 4541, 352: 4542, 369: 4543}, + {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}, + {2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 57: 2178, 458: 4523, 462: 2178, 2178, 2178, 2178, 467: 2178, 475: 2178, 478: 2178, 563: 2178, 571: 2178, 573: 2178, 629: 2178, 2178, 2178, 633: 2178, 1066: 4524, 4525, 1231: 4522}, {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, 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}, + {641: 4539, 731: 4528}, // 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, 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, 1088: 4528}, + {2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 57: 2177, 458: 4537, 462: 2177, 2177, 2177, 2177, 467: 2177, 475: 2177, 478: 2177, 563: 2177, 571: 2177, 573: 2177, 629: 2177, 2177, 2177, 633: 2177, 1067: 4538}, + {2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 57: 2176, 458: 4526, 462: 2176, 2176, 2176, 2176, 467: 2176, 475: 2176, 478: 2176, 563: 2176, 571: 2176, 573: 2176, 629: 2176, 2176, 2176, 633: 2176, 1066: 4527}, + {731: 4528}, + {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}, + {81: 4533, 498: 4532, 657: 4531, 659: 4530, 1089: 4529}, // 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}, + {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}, + {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}, {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}, - {2171, 2171, 2171, 2171, 2171, 2171, 2171, 2171, 2171, 2171, 2171, 2171, 2171, 57: 2171, 458: 2171, 462: 2171, 2171, 2171, 2171, 467: 2171, 475: 2171, 478: 2171, 563: 2171, 571: 2171, 573: 2171, 629: 2171, 2171, 2171, 633: 2171}, - {464: 4535, 478: 4534}, - {289: 4533}, + {464: 4536, 478: 4535}, + {289: 4534}, // 2085 - {2169, 2169, 2169, 2169, 2169, 2169, 2169, 2169, 2169, 2169, 2169, 2169, 2169, 57: 2169, 458: 2169, 462: 2169, 2169, 2169, 2169, 467: 2169, 475: 2169, 478: 2169, 563: 2169, 571: 2169, 573: 2169, 629: 2169, 2169, 2169, 633: 2169}, {2170, 2170, 2170, 2170, 2170, 2170, 2170, 2170, 2170, 2170, 2170, 2170, 2170, 57: 2170, 458: 2170, 462: 2170, 2170, 2170, 2170, 467: 2170, 475: 2170, 478: 2170, 563: 2170, 571: 2170, 573: 2170, 629: 2170, 2170, 2170, 633: 2170}, - {2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 57: 2168, 458: 2168, 462: 2168, 2168, 2168, 2168, 467: 2168, 475: 2168, 478: 2168, 563: 2168, 571: 2168, 573: 2168, 629: 2168, 2168, 2168, 633: 2168}, - {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}, + {2171, 2171, 2171, 2171, 2171, 2171, 2171, 2171, 2171, 2171, 2171, 2171, 2171, 57: 2171, 458: 2171, 462: 2171, 2171, 2171, 2171, 467: 2171, 475: 2171, 478: 2171, 563: 2171, 571: 2171, 573: 2171, 629: 2171, 2171, 2171, 633: 2171}, + {2169, 2169, 2169, 2169, 2169, 2169, 2169, 2169, 2169, 2169, 2169, 2169, 2169, 57: 2169, 458: 2169, 462: 2169, 2169, 2169, 2169, 467: 2169, 475: 2169, 478: 2169, 563: 2169, 571: 2169, 573: 2169, 629: 2169, 2169, 2169, 633: 2169}, + {641: 4539}, + {2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 57: 2175, 458: 2175, 462: 2175, 2175, 2175, 2175, 467: 2175, 475: 2175, 478: 2175, 563: 2175, 571: 2175, 573: 2175, 629: 2175, 2175, 2175, 633: 2175}, // 2090 - {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}, + {81: 4533, 498: 4532, 657: 4531, 659: 4530, 1089: 4540}, + {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}, + {2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 57: 2186, 458: 2186, 462: 2186, 2186, 2186, 2186, 467: 2186, 475: 2186, 478: 2186, 563: 2186, 571: 2186, 573: 2186, 629: 2186, 2186, 2186, 633: 2186}, {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}, - {2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 57: 2183, 458: 2183, 462: 2183, 2183, 2183, 2183, 467: 2183, 475: 2183, 478: 2183, 563: 2183, 571: 2183, 573: 2183, 629: 2183, 2183, 2183, 633: 2183}, // 2095 - {2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 57: 2194, 458: 2194, 462: 2194, 2194, 2194, 2194, 467: 2194, 475: 2194, 478: 2194, 563: 2194, 571: 2194, 573: 2194, 629: 2194, 2194, 2194, 633: 2194}, - {465: 2199}, - {2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 57: 2207, 458: 2207, 462: 2207, 2207, 2207, 2207, 467: 2207, 475: 2207, 478: 2207, 563: 2207, 571: 2207, 573: 2207, 629: 2207, 2207, 2207, 633: 2207}, + {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}, + {465: 2200}, {2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 57: 2208, 458: 2208, 462: 2208, 2208, 2208, 2208, 467: 2208, 475: 2208, 478: 2208, 563: 2208, 571: 2208, 573: 2208, 629: 2208, 2208, 2208, 633: 2208}, - {2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 57: 2206, 458: 2206, 462: 2206, 2206, 2206, 2206, 467: 2206, 475: 2206, 478: 2206, 563: 2206, 571: 2206, 573: 2206, 629: 2206, 2206, 2206, 633: 2206}, + {2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 57: 2209, 458: 2209, 462: 2209, 2209, 2209, 2209, 467: 2209, 475: 2209, 478: 2209, 563: 2209, 571: 2209, 573: 2209, 629: 2209, 2209, 2209, 633: 2209}, + {2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 57: 2207, 458: 2207, 462: 2207, 2207, 2207, 2207, 467: 2207, 475: 2207, 478: 2207, 563: 2207, 571: 2207, 573: 2207, 629: 2207, 2207, 2207, 633: 2207}, // 2100 + {2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 57: 2206, 458: 2206, 462: 2206, 2206, 2206, 2206, 467: 2206, 475: 2206, 478: 2206, 563: 2206, 571: 2206, 573: 2206, 629: 2206, 2206, 2206, 633: 2206}, {2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 2205, 57: 2205, 458: 2205, 462: 2205, 2205, 2205, 2205, 467: 2205, 475: 2205, 478: 2205, 563: 2205, 571: 2205, 573: 2205, 629: 2205, 2205, 2205, 633: 2205}, + {2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 57: 2210, 458: 2210, 462: 2210, 2210, 2210, 2210, 467: 2210, 475: 2210, 478: 2210, 563: 2210, 571: 2210, 573: 2210, 629: 2210, 2210, 2210, 633: 2210}, {2204, 2204, 2204, 2204, 2204, 2204, 2204, 2204, 2204, 2204, 2204, 2204, 2204, 57: 2204, 458: 2204, 462: 2204, 2204, 2204, 2204, 467: 2204, 475: 2204, 478: 2204, 563: 2204, 571: 2204, 573: 2204, 629: 2204, 2204, 2204, 633: 2204}, - {2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 57: 2209, 458: 2209, 462: 2209, 2209, 2209, 2209, 467: 2209, 475: 2209, 478: 2209, 563: 2209, 571: 2209, 573: 2209, 629: 2209, 2209, 2209, 633: 2209}, {2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 2203, 57: 2203, 458: 2203, 462: 2203, 2203, 2203, 2203, 467: 2203, 475: 2203, 478: 2203, 563: 2203, 571: 2203, 573: 2203, 629: 2203, 2203, 2203, 633: 2203}, - {2202, 2202, 2202, 2202, 2202, 2202, 2202, 2202, 2202, 2202, 2202, 2202, 2202, 57: 2202, 458: 2202, 462: 2202, 2202, 2202, 2202, 467: 2202, 475: 2202, 478: 2202, 563: 2202, 571: 2202, 573: 2202, 629: 2202, 2202, 2202, 633: 2202}, // 2105 - {2201, 2201, 2201, 2201, 2201, 2201, 2201, 2201, 2201, 2201, 2201, 2201, 2201, 57: 2201, 458: 2201, 462: 2201, 2201, 2201, 2201, 467: 2201, 475: 2201, 478: 2201, 563: 2201, 571: 2201, 573: 2201, 629: 2201, 2201, 2201, 633: 2201}, - {2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 57: 2210, 458: 2210, 462: 2210, 2210, 2210, 2210, 467: 2210, 475: 2210, 478: 2210, 563: 2210, 571: 2210, 573: 2210, 629: 2210, 2210, 2210, 633: 2210}, - {459: 4556}, - {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, 4557}, - {57: 4558, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, + {2202, 2202, 2202, 2202, 2202, 2202, 2202, 2202, 2202, 2202, 2202, 2202, 2202, 57: 2202, 458: 2202, 462: 2202, 2202, 2202, 2202, 467: 2202, 475: 2202, 478: 2202, 563: 2202, 571: 2202, 573: 2202, 629: 2202, 2202, 2202, 633: 2202}, + {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}, + {459: 4557}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 4558}, + {57: 4559, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, // 2110 - {2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 57: 2198, 458: 2198, 462: 2198, 2198, 2198, 2198, 467: 2198, 475: 2198, 478: 2198, 563: 2198, 571: 2198, 573: 2198, 629: 2198, 2198, 2198, 633: 2198, 1282: 4561, 1312: 4560, 4559}, - {2212, 2212, 2212, 2212, 2212, 2212, 2212, 2212, 2212, 2212, 2212, 2212, 2212, 57: 2212, 458: 2212, 462: 2212, 2212, 2212, 2212, 467: 2212, 475: 2212, 478: 2212, 563: 2212, 571: 2212, 573: 2212, 629: 2212, 2212, 2212, 633: 2212}, + {2199, 2199, 2199, 2199, 2199, 2199, 2199, 2199, 2199, 2199, 2199, 2199, 2199, 57: 2199, 458: 2199, 462: 2199, 2199, 2199, 2199, 467: 2199, 475: 2199, 478: 2199, 563: 2199, 571: 2199, 573: 2199, 629: 2199, 2199, 2199, 633: 2199, 1282: 4562, 1312: 4561, 4560}, + {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}, + {2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 57: 2198, 458: 2198, 462: 2198, 2198, 2198, 2198, 467: 2198, 475: 2198, 478: 2198, 563: 2198, 571: 2198, 573: 2198, 629: 2198, 2198, 2198, 633: 2198}, {2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 57: 2197, 458: 2197, 462: 2197, 2197, 2197, 2197, 467: 2197, 475: 2197, 478: 2197, 563: 2197, 571: 2197, 573: 2197, 629: 2197, 2197, 2197, 633: 2197}, - {2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 57: 2196, 458: 2196, 462: 2196, 2196, 2196, 2196, 467: 2196, 475: 2196, 478: 2196, 563: 2196, 571: 2196, 573: 2196, 629: 2196, 2196, 2196, 633: 2196}, - {459: 4563}, + {459: 4564}, // 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, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 4565}, + {57: 4566, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 57: 2229, 146: 4323, 458: 2229, 462: 3971, 3970, 2229, 2229, 467: 2229, 475: 2229, 478: 2229, 563: 2229, 571: 2229, 573: 2229, 629: 2229, 2229, 2229, 633: 2229, 796: 4567, 918: 4568, 1022: 4569, 1181: 4570}, + {146: 4325, 478: 4571}, + {2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 57: 2228, 458: 2228, 462: 2228, 2228, 2228, 2228, 467: 2228, 475: 2228, 478: 2228, 563: 2228, 571: 2228, 573: 2228, 629: 2228, 2228, 2228, 633: 2228}, // 2120 - {2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 57: 2225, 458: 2225, 462: 2225, 2225, 2225, 2225, 467: 2225, 475: 2225, 478: 2225, 563: 2225, 571: 2225, 573: 2225, 629: 2225, 2225, 2225, 633: 2225}, - {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, 1057: 4574, 4575, 4573}, - // 2125 + {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}, {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}, - {459: 4580}, + {575: 4580, 4577, 4578, 4579, 1058: 4575, 4576, 4574}, + // 2125 + {2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 57: 2216, 458: 2216, 462: 2216, 2216, 2216, 2216, 467: 2216, 475: 2216, 478: 2216, 563: 2216, 571: 2216, 573: 2216, 629: 2216, 2216, 2216, 633: 2216}, + {2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 57: 2165, 458: 2165, 462: 2165, 2165, 2165, 2165, 467: 2165, 475: 2165, 478: 2165, 563: 2165, 571: 2165, 573: 2165, 629: 2165, 2165, 2165, 633: 2165}, + {459: 4581}, + {2156, 2156, 2156, 2156, 2156, 2156, 2156, 2156, 2156, 2156, 2156, 2156, 2156, 57: 2156, 458: 2156, 2160, 462: 2156, 2156, 2156, 2156, 467: 2156, 475: 2156, 478: 2156, 563: 2156, 571: 2156, 573: 2156, 629: 2156, 2156, 2156, 633: 2156}, {2155, 2155, 2155, 2155, 2155, 2155, 2155, 2155, 2155, 2155, 2155, 2155, 2155, 57: 2155, 458: 2155, 2159, 462: 2155, 2155, 2155, 2155, 467: 2155, 475: 2155, 478: 2155, 563: 2155, 571: 2155, 573: 2155, 629: 2155, 2155, 2155, 633: 2155}, - {2154, 2154, 2154, 2154, 2154, 2154, 2154, 2154, 2154, 2154, 2154, 2154, 2154, 57: 2154, 458: 2154, 2158, 462: 2154, 2154, 2154, 2154, 467: 2154, 475: 2154, 478: 2154, 563: 2154, 571: 2154, 573: 2154, 629: 2154, 2154, 2154, 633: 2154}, // 2130 - {2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 57: 2153, 458: 2153, 2157, 462: 2153, 2153, 2153, 2153, 467: 2153, 475: 2153, 478: 2153, 563: 2153, 571: 2153, 573: 2153, 629: 2153, 2153, 2153, 633: 2153}, - {459: 2156}, - {57: 4581, 493: 2638, 722: 4582}, - {2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 57: 2163, 458: 2163, 462: 2163, 2163, 2163, 2163, 467: 2163, 475: 2163, 478: 2163, 563: 2163, 571: 2163, 573: 2163, 629: 2163, 2163, 2163, 633: 2163}, - {57: 4583}, + {2154, 2154, 2154, 2154, 2154, 2154, 2154, 2154, 2154, 2154, 2154, 2154, 2154, 57: 2154, 458: 2154, 2158, 462: 2154, 2154, 2154, 2154, 467: 2154, 475: 2154, 478: 2154, 563: 2154, 571: 2154, 573: 2154, 629: 2154, 2154, 2154, 633: 2154}, + {459: 2157}, + {57: 4582, 493: 2639, 722: 4583}, + {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}, + {57: 4584}, // 2135 - {2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 57: 2162, 458: 2162, 462: 2162, 2162, 2162, 2162, 467: 2162, 475: 2162, 478: 2162, 563: 2162, 571: 2162, 573: 2162, 629: 2162, 2162, 2162, 633: 2162}, - {150: 4585}, - {2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 57: 2216, 458: 2216, 462: 2216, 2216, 2216, 2216, 467: 2216, 475: 2216, 478: 2216, 563: 2216, 571: 2216, 573: 2216, 629: 2216, 2216, 2216, 633: 2216}, + {2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 57: 2163, 458: 2163, 462: 2163, 2163, 2163, 2163, 467: 2163, 475: 2163, 478: 2163, 563: 2163, 571: 2163, 573: 2163, 629: 2163, 2163, 2163, 633: 2163}, + {150: 4586}, {2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 57: 2217, 458: 2217, 462: 2217, 2217, 2217, 2217, 467: 2217, 475: 2217, 478: 2217, 563: 2217, 571: 2217, 573: 2217, 629: 2217, 2217, 2217, 633: 2217}, - {2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 57: 2167, 458: 2167, 462: 2167, 2167, 2167, 2167, 467: 2167, 475: 2167, 478: 2167, 563: 2167, 571: 2167, 573: 2167, 629: 2167, 2167, 2167, 633: 2167}, + {2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 57: 2218, 458: 2218, 462: 2218, 2218, 2218, 2218, 467: 2218, 475: 2218, 478: 2218, 563: 2218, 571: 2218, 573: 2218, 629: 2218, 2218, 2218, 633: 2218}, + {2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 57: 2168, 458: 2168, 462: 2168, 2168, 2168, 2168, 467: 2168, 475: 2168, 478: 2168, 563: 2168, 571: 2168, 573: 2168, 629: 2168, 2168, 2168, 633: 2168}, // 2140 + {2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 57: 2167, 458: 2167, 462: 2167, 2167, 2167, 2167, 467: 2167, 475: 2167, 478: 2167, 563: 2167, 571: 2167, 573: 2167, 629: 2167, 2167, 2167, 633: 2167}, {2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 57: 2166, 458: 2166, 462: 2166, 2166, 2166, 2166, 467: 2166, 475: 2166, 478: 2166, 563: 2166, 571: 2166, 573: 2166, 629: 2166, 2166, 2166, 633: 2166}, - {2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 57: 2165, 458: 2165, 462: 2165, 2165, 2165, 2165, 467: 2165, 475: 2165, 478: 2165, 563: 2165, 571: 2165, 573: 2165, 629: 2165, 2165, 2165, 633: 2165}, - {150: 4060}, - {459: 4057}, - {2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 57: 2218, 458: 2218, 462: 2218, 2218, 2218, 2218, 467: 2218, 475: 2218, 478: 2218, 563: 2218, 571: 2218, 573: 2218, 629: 2218, 2218, 2218, 633: 2218}, + {150: 4061}, + {459: 4058}, + {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: 2219, 571: 2219, 573: 2219, 629: 2219, 2219, 2219, 633: 2219}, // 2145 - {2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 57: 2221, 94: 4594, 96: 4595, 458: 2221, 462: 2221, 2221, 2221, 2221, 467: 2221, 475: 2221, 478: 2221, 563: 2221, 571: 2221, 573: 2221, 629: 2221, 2221, 2221, 633: 2221, 849: 4596}, + {2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 57: 2222, 94: 4595, 96: 4596, 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, 849: 4597}, + {2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 29: 2350, 57: 2350, 92: 2350, 2350, 2350, 2350, 2350, 2350, 458: 2350, 460: 2350, 462: 2350, 2350, 2350, 2350, 467: 2350, 2350, 475: 2350, 478: 2350, 483: 2350, 563: 2350, 571: 2350, 573: 2350, 629: 2350, 2350, 2350, 633: 2350}, {2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 29: 2349, 57: 2349, 92: 2349, 2349, 2349, 2349, 2349, 2349, 458: 2349, 460: 2349, 462: 2349, 2349, 2349, 2349, 467: 2349, 2349, 475: 2349, 478: 2349, 483: 2349, 563: 2349, 571: 2349, 573: 2349, 629: 2349, 2349, 2349, 633: 2349}, - {2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 29: 2348, 57: 2348, 92: 2348, 2348, 2348, 2348, 2348, 2348, 458: 2348, 460: 2348, 462: 2348, 2348, 2348, 2348, 467: 2348, 2348, 475: 2348, 478: 2348, 483: 2348, 563: 2348, 571: 2348, 573: 2348, 629: 2348, 2348, 2348, 633: 2348}, - {2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 57: 2220, 458: 2220, 462: 2220, 2220, 2220, 2220, 467: 2220, 475: 2220, 478: 2220, 563: 2220, 571: 2220, 573: 2220, 629: 2220, 2220, 2220, 633: 2220}, - {2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 57: 2224, 458: 2224, 462: 2224, 2224, 2224, 2224, 467: 2224, 475: 2224, 478: 2224, 563: 2224, 571: 2224, 573: 2224, 629: 2224, 2224, 2224, 633: 2224}, + {2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 57: 2221, 458: 2221, 462: 2221, 2221, 2221, 2221, 467: 2221, 475: 2221, 478: 2221, 563: 2221, 571: 2221, 573: 2221, 629: 2221, 2221, 2221, 633: 2221}, + {2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 57: 2225, 458: 2225, 462: 2225, 2225, 2225, 2225, 467: 2225, 475: 2225, 478: 2225, 563: 2225, 571: 2225, 573: 2225, 629: 2225, 2225, 2225, 633: 2225}, // 2150 - {563: 2321, 571: 2321, 573: 2321, 629: 2321, 635: 2321, 662: 2321, 2321}, - {2320, 2320, 2320, 2320, 7: 2320, 475: 2320, 563: 2320, 571: 2320, 573: 2320, 629: 2320, 635: 2320, 662: 2320, 2320}, - {2257, 2257, 2257, 2257, 7: 2257, 2257, 2257, 57: 2257, 475: 2257}, - {2379, 2379, 2379, 2379, 7: 2379, 475: 2379}, - {2331, 2331, 2331, 2331, 7: 2331, 475: 2331}, + {563: 2322, 571: 2322, 573: 2322, 629: 2322, 635: 2322, 662: 2322, 2322}, + {2321, 2321, 2321, 2321, 7: 2321, 475: 2321, 563: 2321, 571: 2321, 573: 2321, 629: 2321, 635: 2321, 662: 2321, 2321}, + {2258, 2258, 2258, 2258, 7: 2258, 2258, 2258, 57: 2258, 475: 2258}, + {2380, 2380, 2380, 2380, 7: 2380, 475: 2380}, + {2332, 2332, 2332, 2332, 7: 2332, 475: 2332}, // 2155 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4604}, - {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, 913: 4608}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4605}, + {2331, 2331, 2331, 2331, 7: 2331, 475: 2331}, + {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: 4342, 765: 4607}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4346, 832: 4608}, + {2333, 2333, 2333, 2333, 7: 2333, 4603, 4604, 475: 2333, 913: 4609}, // 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, 966: 4613, 1144: 4612}, - {2383, 2383, 2383, 2383, 7: 4616, 475: 2383}, + {2383, 2383, 2383, 2383, 7: 2383, 475: 2383}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4615, 967: 4614, 1144: 4613}, + {2384, 2384, 2384, 2384, 7: 4617, 475: 2384}, // 2165 {1271, 1271, 1271, 1271, 7: 1271, 475: 1271}, - {1261, 1261, 1261, 1261, 7: 1261, 475: 1261, 515: 4512, 4511, 930: 4615}, + {1261, 1261, 1261, 1261, 7: 1261, 475: 1261, 515: 4513, 4512, 930: 4616}, {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, 966: 4617}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4615, 967: 4618}, {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, 774: 4624, 818: 4622}, + {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, 4622, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 638: 550, 805: 4621, 823: 4620}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 638: 4624, 648: 4626, 2665, 2666, 2664, 776: 4625, 818: 4623}, {549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 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}, + {2387, 2387, 2387, 2387, 7: 2387, 475: 2387}, // 2175 - {2355, 2355, 2355, 2355, 7: 2355, 30: 2355, 475: 2355}, - {2354, 2354, 2354, 2354, 7: 4626, 30: 2354, 475: 2354}, - {2325, 2325, 2325, 2325, 7: 2325, 30: 2325, 57: 2325, 98: 2325, 155: 2325, 460: 2325, 475: 2325, 481: 2325, 635: 2325, 638: 2325}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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}, + {2356, 2356, 2356, 2356, 7: 2356, 30: 2356, 475: 2356}, + {2355, 2355, 2355, 2355, 7: 4627, 30: 2355, 475: 2355}, + {2326, 2326, 2326, 2326, 7: 2326, 30: 2326, 57: 2326, 98: 2326, 156: 2326, 460: 2326, 475: 2326, 481: 2326, 635: 2326, 638: 2326}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4628, 2665, 2666, 2664}, + {2325, 2325, 2325, 2325, 7: 2325, 30: 2325, 57: 2325, 98: 2325, 156: 2325, 460: 2325, 475: 2325, 481: 2325, 635: 2325, 638: 2325}, // 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, 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, 774: 4624, 818: 4634}, - // 2185 + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 638: 4624, 648: 4626, 2665, 2666, 2664, 776: 4625, 818: 4631}, {2388, 2388, 2388, 2388, 7: 2388, 475: 2388}, - {30: 4635}, + {30: 4632}, {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, 774: 4624, 818: 4638}, - // 2190 + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 638: 4624, 648: 4626, 2665, 2666, 2664, 776: 4625, 818: 4635}, + // 2185 + {2389, 2389, 2389, 2389, 7: 2389, 475: 2389}, + {30: 4636}, {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, 774: 4624, 818: 4641}, + {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, 4622, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 638: 550, 805: 4621, 823: 4638}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 638: 4624, 648: 4626, 2665, 2666, 2664, 776: 4625, 818: 4639}, + // 2190 {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, 774: 4624, 818: 4643}, - // 2195 + {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, 4622, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 638: 550, 805: 4621, 823: 4641}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 638: 4624, 648: 4626, 2665, 2666, 2664, 776: 4625, 818: 4642}, {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}, - {460: 4646}, - {553: 4647}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4648}, - // 2200 - {2353, 2353, 2353, 2353, 7: 2353, 213: 4652, 460: 4651, 475: 2353, 1323: 4650, 4649}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 638: 4624, 648: 4626, 2665, 2666, 2664, 776: 4625, 818: 4644}, + // 2195 {2394, 2394, 2394, 2394, 7: 2394, 475: 2394}, - {2352, 2352, 2352, 2352, 7: 2352, 475: 2352}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4646, 2665, 2666, 2664}, + {460: 4647}, + {553: 4648}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 4649}, + // 2200 + {2354, 2354, 2354, 2354, 7: 2354, 214: 4653, 460: 4652, 475: 2354, 1323: 4651, 4650}, + {2395, 2395, 2395, 2395, 7: 2395, 475: 2395}, + {2353, 2353, 2353, 2353, 7: 2353, 475: 2353}, + {188: 4655}, {188: 4654}, - {188: 4653}, // 2205 - {2350, 2350, 2350, 2350, 7: 2350, 475: 2350}, {2351, 2351, 2351, 2351, 7: 2351, 475: 2351}, - {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: 4673}, - {563: 4672}, - {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: 4670}, + {2352, 2352, 2352, 2352, 7: 2352, 475: 2352}, + {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: 4342, 765: 4674}, + {563: 4673}, + {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: 4342, 765: 4671}, // 2210 - {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: 4668}, - {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: 4666}, - {563: 4663}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4662, 2664, 2665, 2663}, - {2361, 2361, 2361, 2361, 7: 2361, 475: 2361}, + {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: 4342, 765: 4669}, + {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: 4342, 765: 4667}, + {563: 4664}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4663, 2665, 2666, 2664}, + {2362, 2362, 2362, 2362, 7: 2362, 475: 2362}, // 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, 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}, + {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: 4342, 765: 4665}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4600, 2665, 2666, 2664, 1121: 4666}, {2385, 2385, 2385, 2385, 7: 2385, 475: 2385}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4668, 2665, 2666, 2664}, + {2386, 2386, 2386, 2386, 7: 2386, 475: 2386}, // 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, 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, 894: 4675}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4670, 2665, 2666, 2664}, + {2396, 2396, 2396, 2396, 7: 2396, 475: 2396}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4626, 2665, 2666, 2664, 776: 4672}, + {2397, 2397, 2397, 2397, 7: 4627, 475: 2397}, {2398, 2398, 2398, 2398, 7: 2398, 475: 2398}, + // 2225 + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4675}, + {1993, 1993, 1993, 1993, 7: 1993, 475: 1993, 657: 4678, 659: 4677, 896: 4676}, + {2399, 2399, 2399, 2399, 7: 2399, 475: 2399}, + {1992, 1992, 1992, 1992, 7: 1992, 475: 1992}, {1991, 1991, 1991, 1991, 7: 1991, 475: 1991}, - {1990, 1990, 1990, 1990, 7: 1990, 475: 1990}, // 2230 - {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, 774: 4624, 818: 4682}, + {136: 4622, 493: 550, 805: 4621, 823: 4680}, + {493: 2639, 722: 4681}, {2400, 2400, 2400, 2400, 7: 2400, 475: 2400}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 638: 4624, 648: 4626, 2665, 2666, 2664, 776: 4625, 818: 4683}, + {2401, 2401, 2401, 2401, 7: 2401, 475: 2401}, // 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, 1171: 4694}, + {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: 4702, 769: 4816}, + {2407, 2407, 2407, 2407, 7: 2407, 475: 2407}, + {1805, 1805, 1805, 1805, 7: 1805, 103: 1805, 136: 1805, 459: 1805, 475: 1805, 552: 4702, 769: 4770, 805: 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: 4702, 769: 4761}, + {563: 4300, 571: 4694, 573: 4689, 629: 4692, 635: 4301, 662: 4693, 4690, 814: 4691, 1171: 4695}, // 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, 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, 1045: 4711}, - {563: 4699}, + {563: 4755}, + {2: 2337, 2337, 2337, 2337, 2337, 8: 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 58: 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 459: 2337, 563: 4300, 635: 4301, 814: 4711, 1046: 4749}, + {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: 4702, 769: 4743}, + {2: 2337, 2337, 2337, 2337, 2337, 8: 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 58: 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 2337, 459: 2337, 468: 2337, 563: 4300, 635: 4301, 814: 4711, 1046: 4712}, + {563: 4700}, // 2245 - {459: 4695}, + {459: 4696}, {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, 918: 4567, 1021: 4698}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 4697}, + {57: 4698, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2229, 2229, 2229, 2229, 7: 2229, 57: 2229, 146: 4323, 462: 3971, 3970, 475: 2229, 796: 4324, 918: 4568, 1022: 4699}, // 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, 857: 4704}, - {462: 3970, 3969, 796: 4702}, - {569: 4703}, + {2187, 2187, 2187, 2187, 7: 2187, 57: 2187, 475: 2187}, + {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: 4702, 769: 4701}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 1801, 648: 4706, 2665, 2666, 2664, 859: 4705}, + {462: 3971, 3970, 796: 4703}, + {569: 4704}, // 2255 - {1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 58: 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 459: 1804, 461: 1804, 468: 1804, 475: 1804, 556: 1804, 804: 1804}, - {459: 4706}, + {1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 58: 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 1804, 459: 1804, 461: 1804, 468: 1804, 475: 1804, 556: 1804, 805: 1804}, + {459: 4707}, {459: 1800}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 4508, 648: 4025, 2665, 2666, 2664, 729: 4507, 813: 4506, 822: 4708}, + {7: 4517, 57: 4709}, // 2260 - {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, 857: 4714, 926: 4712}, - {459: 4722}, + {631: 4501, 895: 4710}, + {2188, 2188, 2188, 2188, 7: 2188, 57: 2188, 475: 2188}, + {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 1801, 468: 1801, 648: 4714, 2665, 2666, 2664, 859: 4715, 926: 4713}, + {459: 4723}, // 2265 - {93: 4720, 459: 1800, 468: 1800}, - {459: 1791, 468: 4715}, - {140: 4718, 169: 4717, 182: 4719, 887: 4716}, + {93: 4721, 459: 1800, 468: 1800}, + {459: 1791, 468: 4716}, + {140: 4719, 169: 4718, 182: 4720, 890: 4717}, {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, 887: 4721}, + {140: 4719, 169: 4718, 182: 4720, 890: 4722}, {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 4508, 648: 4025, 2665, 2666, 2664, 729: 4507, 813: 4506, 822: 4724}, // 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, 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}, + {7: 4517, 57: 4725}, + {1799, 1799, 1799, 1799, 1799, 7: 1799, 29: 1799, 57: 1799, 93: 1799, 1799, 1799, 1799, 1799, 460: 1799, 468: 1799, 475: 1799, 861: 4726}, + {2189, 2189, 2189, 2189, 4731, 7: 2189, 29: 4728, 57: 2189, 93: 4735, 4595, 4320, 4596, 4319, 460: 4730, 468: 4734, 475: 2189, 840: 4732, 842: 4729, 849: 4733, 860: 4727}, {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}, + {482: 4172, 493: 1988, 723: 4741}, // 2280 {1796, 1796, 1796, 1796, 1796, 7: 1796, 29: 1796, 57: 1796, 92: 1796, 1796, 1796, 1796, 1796, 1796, 460: 1796, 468: 1796, 475: 1796, 483: 1796}, - {351: 4738}, - {461: 4737}, + {351: 4739}, + {461: 4738}, {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, 887: 4736}, - {140: 4718, 169: 4717, 182: 4719, 887: 4735}, + {140: 4719, 169: 4718, 182: 4720, 890: 4737}, + {140: 4719, 169: 4718, 182: 4720, 890: 4736}, {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}, // 2290 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 4739, 2664, 2665, 2663}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4740, 2665, 2666, 2664}, {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}, + {493: 2639, 722: 2638, 730: 4742}, {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, 857: 4714, 926: 4743}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 1801, 468: 1801, 648: 4714, 2665, 2666, 2664, 859: 4715, 926: 4744}, // 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, 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}, + {459: 4745}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 4508, 648: 4025, 2665, 2666, 2664, 729: 4507, 813: 4506, 822: 4746}, + {7: 4517, 57: 4747}, + {1799, 1799, 1799, 1799, 1799, 7: 1799, 29: 1799, 57: 1799, 93: 1799, 1799, 1799, 1799, 1799, 460: 1799, 468: 1799, 475: 1799, 861: 4748}, + {2190, 2190, 2190, 2190, 4731, 7: 2190, 29: 4728, 57: 2190, 93: 4735, 4595, 4320, 4596, 4319, 460: 4730, 468: 4734, 475: 2190, 840: 4732, 842: 4729, 849: 4733, 860: 4727}, // 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, 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, 859: 4753}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 1801, 648: 4706, 2665, 2666, 2664, 859: 4750}, + {459: 4751}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 4508, 648: 4025, 2665, 2666, 2664, 729: 4507, 813: 4506, 822: 4752}, + {7: 4517, 57: 4753}, + {1799, 1799, 1799, 1799, 1799, 7: 1799, 29: 1799, 57: 1799, 93: 1799, 1799, 1799, 1799, 1799, 460: 1799, 468: 1799, 475: 1799, 861: 4754}, // 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, 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}, + {2191, 2191, 2191, 2191, 4731, 7: 2191, 29: 4728, 57: 2191, 93: 4735, 4595, 4320, 4596, 4319, 460: 4730, 468: 4734, 475: 2191, 840: 4732, 842: 4729, 849: 4733, 860: 4727}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 1801, 468: 1801, 648: 4714, 2665, 2666, 2664, 859: 4715, 926: 4756}, + {459: 4757}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 4508, 648: 4025, 2665, 2666, 2664, 729: 4507, 813: 4506, 822: 4758}, + {7: 4517, 57: 4759}, // 2310 - {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, 1116: 4762}, - {459: 4766}, + {1799, 1799, 1799, 1799, 1799, 7: 1799, 29: 1799, 57: 1799, 93: 1799, 1799, 1799, 1799, 1799, 460: 1799, 468: 1799, 475: 1799, 861: 4760}, + {2192, 2192, 2192, 2192, 4731, 7: 2192, 29: 4728, 57: 2192, 93: 4735, 4595, 4320, 4596, 4319, 460: 4730, 468: 4734, 475: 2192, 840: 4732, 842: 4729, 849: 4733, 860: 4727}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4762, 2665, 2666, 2664}, + {221: 4764, 230: 4766, 233: 4765, 1117: 4763}, + {459: 4767}, // 2315 + {57: 2147, 459: 2147}, {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, 794: 4767}, - {7: 4027, 57: 4768}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4026, 795: 4768}, + {7: 4028, 57: 4769}, // 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, 1076: 4771}, - {2405, 2405, 2405, 2405, 7: 2405, 475: 2405}, - {493: 2638, 722: 4814}, + {2404, 2404, 2404, 2404, 7: 2404, 475: 2404}, + {550, 550, 550, 550, 7: 550, 103: 550, 136: 4622, 459: 550, 475: 550, 805: 4621, 823: 4771}, + {2083, 2083, 2083, 2083, 7: 2083, 103: 4773, 459: 4774, 475: 2083, 1077: 4772}, + {2406, 2406, 2406, 2406, 7: 2406, 475: 2406}, + {493: 2639, 722: 4815}, // 2325 - {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}, + {475: 4777, 933: 4776, 1076: 4775}, + {7: 4813, 57: 4812}, + {7: 2081, 57: 2081}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4778, 2665, 2666, 2664}, + {4: 2060, 2060, 7: 2060, 15: 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 2060, 30: 2060, 2060, 2060, 2060, 2060, 2060, 2060, 57: 2060, 145: 4783, 327: 4782, 459: 2060, 464: 4781, 484: 4780, 635: 2060, 1246: 4779}, // 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, 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: 2073, 2073, 7: 2073, 15: 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 30: 2073, 2073, 2073, 2073, 2073, 2073, 2073, 57: 2073, 459: 2073, 635: 2073, 932: 4799}, + {337: 4784, 530: 4785}, + {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}, + {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}, {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}, - {4: 2053, 2053, 7: 2053, 15: 2053, 2053, 2053, 2053, 2053, 2053, 2053, 2053, 2053, 2053, 2053, 2053, 30: 2053, 2053, 2053, 2053, 2053, 2053, 2053, 57: 2053, 459: 2053, 635: 2053}, // 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, 929: 4789, 1056: 4786}, - {7: 4791, 57: 4790}, + {381: 4794}, + {459: 4786}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 644: 4788, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 4789, 929: 4790, 1057: 4787}, + {7: 4792, 57: 4791}, {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: 1880, 57: 1880, 471: 3574, 3573, 3579, 509: 3575, 542: 3576, 3577, 3570, 3580, 3569, 3578, 3571, 3572}, {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, 929: 4792}, + {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 644: 4788, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 4789, 929: 4793}, {7: 1867, 57: 1867}, // 2345 - {459: 4795, 644: 4794}, + {459: 4796, 644: 4795}, + {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, 459: 2059, 635: 2059}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 644: 4788, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 4789, 929: 4790, 1057: 4797}, + {7: 4792, 57: 4798}, {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, 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, 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}, + {4: 4132, 4803, 7: 2078, 15: 4078, 4090, 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 30: 4140, 4133, 4136, 4135, 4138, 4139, 4141, 57: 2078, 459: 4801, 635: 4137, 758: 4088, 762: 4089, 766: 4142, 798: 4802, 1287: 4800}, + {7: 2079, 57: 2079}, + {100: 4806, 1119: 4805, 1286: 4804}, + {2072, 2072, 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}, + {31: 4271}, // 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, 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: 4810, 57: 4809}, {7: 2076, 57: 2076}, - {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, 933: 4813}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4807, 2665, 2666, 2664}, + {4: 2073, 2073, 7: 2073, 15: 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 30: 2073, 2073, 2073, 2073, 2073, 2073, 2073, 57: 2073, 635: 2073, 932: 4808}, + {4: 4132, 4803, 7: 2074, 15: 4078, 4090, 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 30: 4140, 4133, 4136, 4135, 4138, 4139, 4141, 57: 2074, 635: 4137, 758: 4088, 762: 4089, 766: 4142, 798: 4802}, + // 2360 + {7: 2077, 57: 2077}, + {100: 4806, 1119: 4811}, + {7: 2075, 57: 2075}, + {2082, 2082, 2082, 2082, 7: 2082, 458: 2082, 2082, 2082, 465: 2082, 474: 2082, 2082, 484: 2082, 491: 2082, 553: 2082, 632: 2082}, + {475: 4777, 933: 4814}, // 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, 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}, + {7: 2080, 57: 2080}, + {2405, 2405, 2405, 2405, 7: 2405, 475: 2405}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 4818, 648: 4025, 2665, 2666, 2664, 729: 4346, 832: 4817}, + {2333, 2333, 2333, 2333, 7: 2333, 4603, 4604, 475: 2333, 913: 4826}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 563: 2324, 571: 2324, 573: 2324, 629: 2324, 4479, 635: 2324, 648: 4025, 2665, 2666, 2664, 662: 2324, 2324, 729: 4346, 819: 4688, 832: 4820, 888: 4821, 949: 4822, 1122: 4819}, // 2370 - {7: 4823, 57: 4822}, + {7: 4824, 57: 4823}, {7: 429, 57: 429}, {7: 428, 57: 428}, {7: 427, 57: 427}, - {2407, 2407, 2407, 2407, 7: 2407, 475: 2407}, + {2408, 2408, 2408, 2408, 7: 2408, 475: 2408}, // 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, 885: 4820, 949: 4824}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 563: 2324, 571: 2324, 573: 2324, 629: 2324, 4479, 635: 2324, 648: 4025, 2665, 2666, 2664, 662: 2324, 2324, 729: 4346, 819: 4688, 832: 4820, 888: 4821, 949: 4825}, {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, 793: 4828}, - // 2380 - {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}, + {13: 3721, 486: 3722, 634: 3720, 759: 4828}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 464: 4830, 529: 3646, 648: 3378, 2665, 2666, 2664, 725: 3645, 794: 4829}, + // 2380 + {257, 257, 257, 257, 7: 257, 467: 4832, 475: 257, 1068: 4834}, + {257, 257, 257, 257, 7: 257, 467: 4832, 475: 257, 1068: 4831}, + {2410, 2410, 2410, 2410, 7: 2410, 475: 2410}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 529: 3376, 648: 3378, 2665, 2666, 2664, 725: 3375, 853: 4833}, {256, 256, 256, 256, 7: 256, 475: 256}, // 2385 - {2410, 2410, 2410, 2410, 7: 2410, 475: 2410}, - {364: 4835}, - {493: 2638, 722: 2637, 730: 4836}, - {2414, 2414, 2414, 2414, 7: 2414, 199: 4837, 475: 2414, 1221: 4838}, - {251: 4839}, - // 2390 {2411, 2411, 2411, 2411, 7: 2411, 475: 2411}, - {461: 4841, 1283: 4840}, - {2413, 2413, 2413, 2413, 7: 4842, 475: 2413}, + {364: 4836}, + {493: 2639, 722: 2638, 730: 4837}, + {2415, 2415, 2415, 2415, 7: 2415, 199: 4838, 475: 2415, 1221: 4839}, + {252: 4840}, + // 2390 + {2412, 2412, 2412, 2412, 7: 2412, 475: 2412}, + {461: 4842, 1283: 4841}, + {2414, 2414, 2414, 2414, 7: 4843, 475: 2414}, {255, 255, 255, 255, 7: 255, 475: 255}, - {461: 4843}, + {461: 4844}, // 2395 {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, 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}, + {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, 491: 382, 553: 382, 632: 382, 634: 382, 382}, + {4: 4132, 4134, 389, 13: 2107, 4151, 4078, 4090, 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 4149, 4169, 4153, 4140, 4133, 4136, 4135, 4138, 4139, 4141, 4148, 389, 4159, 4160, 4146, 4147, 4152, 4154, 4166, 4165, 4171, 4167, 4164, 4157, 4162, 4163, 4156, 4158, 4161, 4150, 464: 4131, 466: 4168, 2107, 485: 4845, 2107, 634: 2107, 4137, 758: 4088, 762: 4089, 766: 4142, 779: 4144, 798: 4143, 821: 4145, 825: 4155, 829: 4848}, + {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, 491: 381, 553: 381, 632: 381, 634: 381, 381}, // 2400 - {461: 4850, 464: 4849}, + {461: 4851, 464: 4850}, + {2422, 2422, 2422, 2422, 7: 2422, 475: 2422}, {2421, 2421, 2421, 2421, 7: 2421, 475: 2421}, - {2420, 2420, 2420, 2420, 7: 2420, 475: 2420}, - {461: 4853, 464: 4852}, - {2423, 2423, 2423, 2423, 7: 2423, 475: 2423}, + {461: 4854, 464: 4853}, + {2424, 2424, 2424, 2424, 7: 2424, 475: 2424}, // 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, 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}, + {2423, 2423, 2423, 2423, 7: 2423, 475: 2423}, + {2: 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, 461: 1988, 464: 1988, 482: 4172, 498: 4857, 723: 4856}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 4859, 464: 4861, 648: 4862, 2665, 2666, 2664, 868: 4860}, + {464: 4858}, + {2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 13: 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 458: 2425, 2425, 2425, 464: 2425, 2425, 2425, 2425, 474: 2425, 2425, 484: 2425, 2425, 2425, 491: 2425, 553: 2425, 632: 2425, 634: 2425, 2425}, // 2410 - {2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 13: 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 458: 2427, 2427, 2427, 464: 2427, 2427, 2427, 2427, 474: 2427, 2427, 484: 2427, 2427, 2427, 492: 2427, 553: 2427, 632: 2427, 634: 2427, 2427}, - {2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 13: 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 458: 2426, 2426, 2426, 464: 2426, 2426, 2426, 2426, 474: 2426, 2426, 484: 2426, 2426, 2426, 492: 2426, 553: 2426, 632: 2426, 634: 2426, 2426}, - {2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 13: 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 2425, 458: 2425, 2425, 2425, 464: 2425, 2425, 2425, 2425, 474: 2425, 2425, 484: 2425, 2425, 2425, 492: 2425, 553: 2425, 632: 2425, 634: 2425, 2425}, - {2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 13: 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 458: 2121, 2121, 2121, 464: 2121, 2121, 2121, 2121, 474: 2121, 2121, 484: 2121, 2121, 2121, 492: 2121, 553: 2121, 632: 2121, 634: 2121, 2121}, - {461: 4863}, + {2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 13: 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 2428, 458: 2428, 2428, 2428, 464: 2428, 2428, 2428, 2428, 474: 2428, 2428, 484: 2428, 2428, 2428, 491: 2428, 553: 2428, 632: 2428, 634: 2428, 2428}, + {2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 13: 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 2427, 458: 2427, 2427, 2427, 464: 2427, 2427, 2427, 2427, 474: 2427, 2427, 484: 2427, 2427, 2427, 491: 2427, 553: 2427, 632: 2427, 634: 2427, 2427}, + {2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 13: 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 2426, 458: 2426, 2426, 2426, 464: 2426, 2426, 2426, 2426, 474: 2426, 2426, 484: 2426, 2426, 2426, 491: 2426, 553: 2426, 632: 2426, 634: 2426, 2426}, + {2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 13: 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 2122, 458: 2122, 2122, 2122, 464: 2122, 2122, 2122, 2122, 474: 2122, 2122, 484: 2122, 2122, 2122, 491: 2122, 553: 2122, 632: 2122, 634: 2122, 2122}, + {461: 4864}, // 2415 - {2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 13: 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 2430, 458: 2430, 2430, 2430, 464: 2430, 2430, 2430, 2430, 474: 2430, 2430, 484: 2430, 2430, 2430, 492: 2430, 553: 2430, 632: 2430, 634: 2430, 2430}, - {461: 4865}, - {2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 13: 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 458: 2431, 2431, 2431, 464: 2431, 2431, 2431, 2431, 474: 2431, 2431, 484: 2431, 2431, 2431, 492: 2431, 553: 2431, 632: 2431, 634: 2431, 2431}, - {461: 4867}, - {2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 13: 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 458: 2432, 2432, 2432, 464: 2432, 2432, 2432, 2432, 474: 2432, 2432, 484: 2432, 2432, 2432, 492: 2432, 553: 2432, 632: 2432, 634: 2432, 2432}, + {2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 13: 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 2431, 458: 2431, 2431, 2431, 464: 2431, 2431, 2431, 2431, 474: 2431, 2431, 484: 2431, 2431, 2431, 491: 2431, 553: 2431, 632: 2431, 634: 2431, 2431}, + {461: 4866}, + {2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 13: 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 2432, 458: 2432, 2432, 2432, 464: 2432, 2432, 2432, 2432, 474: 2432, 2432, 484: 2432, 2432, 2432, 491: 2432, 553: 2432, 632: 2432, 634: 2432, 2432}, + {461: 4868}, + {2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 13: 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 458: 2433, 2433, 2433, 464: 2433, 2433, 2433, 2433, 474: 2433, 2433, 484: 2433, 2433, 2433, 491: 2433, 553: 2433, 632: 2433, 634: 2433, 2433}, // 2420 - {461: 4869}, - {2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 13: 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 2433, 458: 2433, 2433, 2433, 464: 2433, 2433, 2433, 2433, 474: 2433, 2433, 484: 2433, 2433, 2433, 492: 2433, 553: 2433, 632: 2433, 634: 2433, 2433}, - {461: 4871}, - {2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 13: 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 458: 2434, 2434, 2434, 464: 2434, 2434, 2434, 2434, 474: 2434, 2434, 484: 2434, 2434, 2434, 492: 2434, 553: 2434, 632: 2434, 634: 2434, 2434}, - {461: 4873}, + {461: 4870}, + {2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 13: 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 2434, 458: 2434, 2434, 2434, 464: 2434, 2434, 2434, 2434, 474: 2434, 2434, 484: 2434, 2434, 2434, 491: 2434, 553: 2434, 632: 2434, 634: 2434, 2434}, + {461: 4872}, + {2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 13: 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 458: 2435, 2435, 2435, 464: 2435, 2435, 2435, 2435, 474: 2435, 2435, 484: 2435, 2435, 2435, 491: 2435, 553: 2435, 632: 2435, 634: 2435, 2435}, + {461: 4874}, // 2425 - {2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 13: 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 2435, 458: 2435, 2435, 2435, 464: 2435, 2435, 2435, 2435, 474: 2435, 2435, 484: 2435, 2435, 2435, 492: 2435, 553: 2435, 632: 2435, 634: 2435, 2435}, - {493: 2638, 722: 2637, 730: 4875}, - {2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 13: 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 458: 2436, 2436, 2436, 464: 2436, 2436, 2436, 2436, 474: 2436, 2436, 484: 2436, 2436, 2436, 492: 2436, 553: 2436, 632: 2436, 634: 2436, 2436}, - {493: 2638, 722: 2637, 730: 4877}, - {2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 13: 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 458: 2437, 2437, 2437, 464: 2437, 2437, 2437, 2437, 474: 2437, 2437, 484: 2437, 2437, 2437, 492: 2437, 553: 2437, 632: 2437, 634: 2437, 2437}, + {2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 13: 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 458: 2436, 2436, 2436, 464: 2436, 2436, 2436, 2436, 474: 2436, 2436, 484: 2436, 2436, 2436, 491: 2436, 553: 2436, 632: 2436, 634: 2436, 2436}, + {493: 2639, 722: 2638, 730: 4876}, + {2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 13: 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 458: 2437, 2437, 2437, 464: 2437, 2437, 2437, 2437, 474: 2437, 2437, 484: 2437, 2437, 2437, 491: 2437, 553: 2437, 632: 2437, 634: 2437, 2437}, + {493: 2639, 722: 2638, 730: 4878}, + {2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 13: 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 458: 2438, 2438, 2438, 464: 2438, 2438, 2438, 2438, 474: 2438, 2438, 484: 2438, 2438, 2438, 491: 2438, 553: 2438, 632: 2438, 634: 2438, 2438}, // 2430 - {493: 2638, 722: 2637, 730: 4879}, - {2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 13: 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 458: 2438, 2438, 2438, 464: 2438, 2438, 2438, 2438, 474: 2438, 2438, 484: 2438, 2438, 2438, 492: 2438, 553: 2438, 632: 2438, 634: 2438, 2438}, - {461: 4881}, - {2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 13: 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 458: 2439, 2439, 2439, 464: 2439, 2439, 2439, 2439, 474: 2439, 2439, 484: 2439, 2439, 2439, 492: 2439, 553: 2439, 632: 2439, 634: 2439, 2439}, - {461: 4883}, + {493: 2639, 722: 2638, 730: 4880}, + {2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 13: 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 458: 2439, 2439, 2439, 464: 2439, 2439, 2439, 2439, 474: 2439, 2439, 484: 2439, 2439, 2439, 491: 2439, 553: 2439, 632: 2439, 634: 2439, 2439}, + {461: 4882}, + {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, 491: 2440, 553: 2440, 632: 2440, 634: 2440, 2440}, + {461: 4884}, // 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, 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}, + {2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 13: 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 458: 2441, 2441, 2441, 464: 2441, 2441, 2441, 2441, 474: 2441, 2441, 484: 2441, 2441, 2441, 491: 2441, 553: 2441, 632: 2441, 634: 2441, 2441}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4626, 2665, 2666, 2664, 776: 4886}, + {2285, 2285, 7: 4627, 460: 4889, 635: 4888, 790: 4887}, + {2446, 2446}, + {873, 873, 2901, 2749, 2785, 2903, 2676, 873, 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 460: 873, 573: 4906, 648: 4905, 2665, 2666, 2664, 841: 4904}, // 2440 - {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}, - {184: 4894}, + {493: 4894, 559: 3312, 3311, 722: 4892, 806: 4893, 973: 4891, 1149: 4890}, + {2284, 2284, 7: 4902}, + {2283, 2283, 7: 2283}, + {220: 4896, 224: 4898, 268: 4899, 287: 4897}, + {184: 4895}, // 2445 - {184: 2149, 219: 1920, 223: 1920, 268: 1920, 287: 1920}, - {2275, 2275, 7: 2275}, + {184: 2150, 220: 1920, 224: 1920, 268: 1920, 287: 1920}, + {2276, 2276, 7: 2276}, + {2281, 2281, 7: 2281}, {2280, 2280, 7: 2280}, - {2279, 2279, 7: 2279}, - {313: 4899, 392: 4900}, + {313: 4900, 392: 4901}, // 2450 - {2276, 2276, 7: 2276}, - {2278, 2278, 7: 2278}, {2277, 2277, 7: 2277}, - {493: 4893, 559: 3311, 3310, 722: 4891, 805: 4892, 972: 4902}, - {2281, 2281, 7: 2281}, + {2279, 2279, 7: 2279}, + {2278, 2278, 7: 2278}, + {493: 4894, 559: 3312, 3311, 722: 4892, 806: 4893, 973: 4903}, + {2282, 2282, 7: 2282}, // 2455 - {2284, 2284, 7: 4907, 460: 4888, 788: 4906}, + {2285, 2285, 7: 4908, 460: 4889, 790: 4907}, {872, 872, 7: 872, 57: 872, 460: 872}, {870, 870, 7: 870, 57: 870, 460: 870}, - {2444, 2444}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 573: 4909, 648: 4908, 2664, 2665, 2663}, + {2445, 2445}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 573: 4910, 648: 4909, 2665, 2666, 2664}, // 2460 {871, 871, 7: 871, 57: 871, 460: 871}, {869, 869, 7: 869, 57: 869, 460: 869}, - {2446, 2446}, - {2419, 2419}, - {353: 4977}, + {2447, 2447}, + {2420, 2420}, + {353: 4978}, // 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, 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}, + {475: 4970}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 646: 4917, 648: 4916, 2665, 2666, 2664}, + {2073, 2073, 4: 2073, 2073, 15: 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 30: 2073, 2073, 2073, 2073, 2073, 2073, 2073, 190: 4091, 635: 2073, 910: 4968, 932: 4969}, + {140: 2091, 339: 4922, 377: 4923, 513: 4921, 563: 2091, 1051: 4924, 4919, 1120: 4920, 1248: 4918}, + {2085, 2085, 100: 2085, 103: 4958, 458: 2085, 2085, 2085, 465: 2085, 474: 2085, 484: 2085, 491: 2085, 553: 2085, 632: 2085, 1249: 4957}, // 2470 - {140: 4944, 563: 4943}, - {2098, 2098, 100: 2098, 103: 2098, 458: 2098, 2098, 2098, 465: 2098, 474: 2098, 484: 2098, 492: 2098, 553: 2098, 632: 2098}, - {98: 3828, 107: 3827, 459: 4936, 820: 4937}, - {98: 3828, 107: 3827, 459: 4929, 820: 4930}, - {2091, 2091, 100: 2091, 103: 2091, 458: 2091, 2091, 2091, 465: 2091, 474: 2091, 480: 4925, 484: 2091, 492: 2091, 553: 2091, 567: 4924, 632: 2091}, + {140: 4945, 563: 4944}, + {2099, 2099, 100: 2099, 103: 2099, 458: 2099, 2099, 2099, 465: 2099, 474: 2099, 484: 2099, 491: 2099, 553: 2099, 632: 2099}, + {98: 3829, 107: 3828, 459: 4937, 820: 4938}, + {98: 3829, 107: 3828, 459: 4930, 820: 4931}, + {2092, 2092, 100: 2092, 103: 2092, 458: 2092, 2092, 2092, 465: 2092, 474: 2092, 480: 4926, 484: 2092, 491: 2092, 553: 2092, 567: 4925, 632: 2092}, // 2475 - {140: 2089, 563: 2089}, - {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, 792: 3330, 816: 4928}, + {140: 2090, 563: 2090}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 4928}, + {493: 2639, 722: 2638, 730: 4927}, + {2093, 2093, 100: 2093, 103: 2093, 458: 2093, 2093, 2093, 465: 2093, 474: 2093, 484: 2093, 491: 2093, 553: 2093, 632: 2093}, + {105: 3351, 3347, 108: 3344, 3359, 111: 3346, 3343, 3345, 3349, 3350, 3355, 3354, 3353, 3357, 3358, 3352, 3356, 3348, 492: 3235, 494: 3233, 3234, 3232, 3230, 517: 3341, 3338, 3340, 3339, 3335, 3337, 3336, 3333, 3334, 3332, 3342, 720: 3231, 3229, 793: 3331, 816: 4929}, // 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, 794: 4932}, - {7: 4027, 57: 4933}, + {2094, 2094, 100: 2094, 103: 2094, 458: 2094, 2094, 2094, 465: 2094, 474: 2094, 484: 2094, 491: 2094, 553: 2094, 632: 2094}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 4935}, + {459: 4932}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4026, 795: 4933}, + {7: 4028, 57: 4934}, // 2485 - {2094, 2094, 100: 2094, 103: 2094, 458: 2094, 2094, 2094, 465: 2094, 474: 2094, 484: 2094, 492: 2094, 553: 2094, 632: 2094}, - {57: 4935, 471: 3573, 3572, 3578, 509: 3574, 542: 3575, 3576, 3569, 3579, 3568, 3577, 3570, 3571}, - {2095, 2095, 100: 2095, 103: 2095, 458: 2095, 2095, 2095, 465: 2095, 474: 2095, 484: 2095, 492: 2095, 553: 2095, 632: 2095}, - {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}, + {2095, 2095, 100: 2095, 103: 2095, 458: 2095, 2095, 2095, 465: 2095, 474: 2095, 484: 2095, 491: 2095, 553: 2095, 632: 2095}, + {57: 4936, 471: 3574, 3573, 3579, 509: 3575, 542: 3576, 3577, 3570, 3580, 3569, 3578, 3571, 3572}, + {2096, 2096, 100: 2096, 103: 2096, 458: 2096, 2096, 2096, 465: 2096, 474: 2096, 484: 2096, 491: 2096, 553: 2096, 632: 2096}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 4942}, + {459: 4939}, // 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, 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}, - {2097, 2097, 100: 2097, 103: 2097, 458: 2097, 2097, 2097, 465: 2097, 474: 2097, 484: 2097, 492: 2097, 553: 2097, 632: 2097}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4026, 795: 4940}, + {7: 4028, 57: 4941}, + {2097, 2097, 100: 2097, 103: 2097, 458: 2097, 2097, 2097, 465: 2097, 474: 2097, 484: 2097, 491: 2097, 553: 2097, 632: 2097}, + {57: 4943, 471: 3574, 3573, 3579, 509: 3575, 542: 3576, 3577, 3570, 3580, 3569, 3578, 3571, 3572}, + {2098, 2098, 100: 2098, 103: 2098, 458: 2098, 2098, 2098, 465: 2098, 474: 2098, 484: 2098, 491: 2098, 553: 2098, 632: 2098}, // 2495 - {92: 4949, 459: 2100, 1247: 4948}, - {459: 4945}, - {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, 4946}, - {57: 4947, 471: 3573, 3572, 3578, 509: 3574, 542: 3575, 3576, 3569, 3579, 3568, 3577, 3570, 3571}, - {2101, 2101, 100: 2101, 103: 2101, 210: 2101, 458: 2101, 2101, 2101, 465: 2101, 474: 2101, 484: 2101, 492: 2101, 553: 2101, 632: 2101}, + {92: 4950, 459: 2101, 1247: 4949}, + {459: 4946}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 4947}, + {57: 4948, 471: 3574, 3573, 3579, 509: 3575, 542: 3576, 3577, 3570, 3580, 3569, 3578, 3571, 3572}, + {2102, 2102, 100: 2102, 103: 2102, 211: 2102, 458: 2102, 2102, 2102, 465: 2102, 474: 2102, 484: 2102, 491: 2102, 553: 2102, 632: 2102}, // 2500 - {459: 4952}, - {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, 794: 4953, 982: 4954}, + {459: 4953}, + {482: 4951}, + {493: 2639, 722: 4952}, + {459: 2100}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 2251, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4026, 795: 4954, 983: 4955}, // 2505 - {7: 4027, 57: 2249}, - {57: 4955}, - {2102, 2102, 100: 2102, 103: 2102, 210: 2102, 458: 2102, 2102, 2102, 465: 2102, 474: 2102, 484: 2102, 492: 2102, 553: 2102, 632: 2102}, - {2088, 2088, 100: 4960, 458: 2088, 2088, 2088, 465: 2088, 474: 2088, 484: 2088, 492: 2088, 553: 2088, 632: 2088, 1289: 4959}, - {493: 2638, 722: 2637, 730: 4958}, + {7: 4028, 57: 2250}, + {57: 4956}, + {2103, 2103, 100: 2103, 103: 2103, 211: 2103, 458: 2103, 2103, 2103, 465: 2103, 474: 2103, 484: 2103, 491: 2103, 553: 2103, 632: 2103}, + {2089, 2089, 100: 4961, 458: 2089, 2089, 2089, 465: 2089, 474: 2089, 484: 2089, 491: 2089, 553: 2089, 632: 2089, 1289: 4960}, + {493: 2639, 722: 2638, 730: 4959}, // 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, 1076: 4966}, - {646: 4961}, - {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}, + {2084, 2084, 100: 2084, 458: 2084, 2084, 2084, 465: 2084, 474: 2084, 484: 2084, 491: 2084, 553: 2084, 632: 2084}, + {2083, 2083, 458: 2083, 4774, 2083, 465: 2083, 474: 2083, 484: 2083, 491: 2083, 553: 2083, 632: 2083, 1077: 4967}, + {646: 4962}, + {140: 2091, 563: 2091, 1051: 4924, 4919, 1120: 4963}, + {2087, 2087, 211: 4965, 458: 2087, 2087, 2087, 465: 2087, 474: 2087, 484: 2087, 491: 2087, 553: 2087, 632: 2087, 1288: 4964}, // 2515 - {2087, 2087, 458: 2087, 2087, 2087, 465: 2087, 474: 2087, 484: 2087, 492: 2087, 553: 2087, 632: 2087}, - {493: 2638, 722: 2637, 730: 4965}, - {2085, 2085, 458: 2085, 2085, 2085, 465: 2085, 474: 2085, 484: 2085, 492: 2085, 553: 2085, 632: 2085}, - {2103, 2103, 458: 2103, 2103, 2103, 465: 2103, 474: 2103, 484: 2103, 492: 2103, 553: 2103, 632: 2103}, - {2416, 2416}, - // 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, 774: 4972, 1257: 4971}, + {2088, 2088, 458: 2088, 2088, 2088, 465: 2088, 474: 2088, 484: 2088, 491: 2088, 553: 2088, 632: 2088}, + {493: 2639, 722: 2638, 730: 4966}, + {2086, 2086, 458: 2086, 2086, 2086, 465: 2086, 474: 2086, 484: 2086, 491: 2086, 553: 2086, 632: 2086}, + {2104, 2104, 458: 2104, 2104, 2104, 465: 2104, 474: 2104, 484: 2104, 491: 2104, 553: 2104, 632: 2104}, {2417, 2417}, - {7: 4626, 481: 4973}, - // 2525 - {459: 4974}, - {475: 4776, 933: 4775, 1075: 4975}, - {7: 4812, 57: 4976}, - {2356, 2356}, + // 2520 + {2416, 2416, 4: 4132, 4803, 15: 4078, 4090, 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 30: 4140, 4133, 4136, 4135, 4138, 4139, 4141, 635: 4137, 758: 4088, 762: 4089, 766: 4142, 798: 4802}, + {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, 4622, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 805: 4621, 823: 4971}, + {2358, 2358, 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4626, 2665, 2666, 2664, 776: 4973, 1257: 4972}, {2418, 2418}, + {7: 4627, 481: 4974}, + // 2525 + {459: 4975}, + {475: 4777, 933: 4776, 1076: 4976}, + {7: 4813, 57: 4977}, + {2357, 2357}, + {2419, 2419}, // 2530 - {136: 4979, 860: 96, 1054: 4980}, - {860: 95}, - {860: 4981}, - {461: 4982}, - {17, 17, 175: 17, 341: 4984, 645: 17, 1227: 4983}, + {136: 4980, 862: 96, 1055: 4981}, + {862: 95}, + {862: 4982}, + {461: 4983}, + {17, 17, 175: 17, 341: 4985, 645: 17, 1227: 4984}, // 2535 - {15, 15, 175: 4987, 645: 15, 1226: 4986}, - {493: 2638, 722: 4985}, + {15, 15, 175: 4988, 645: 15, 1226: 4987}, + {493: 2639, 722: 4986}, {16, 16, 175: 16, 645: 16}, - {81, 81, 645: 3849, 928: 4994}, - {13, 13, 179: 13, 354: 4989, 645: 13, 1251: 4988}, + {81, 81, 645: 3850, 928: 4995}, + {13, 13, 179: 13, 354: 4990, 645: 13, 1251: 4989}, // 2540 - {11, 11, 179: 4992, 645: 11, 1250: 4991}, - {493: 2638, 722: 4990}, + {11, 11, 179: 4993, 645: 11, 1250: 4992}, + {493: 2639, 722: 4991}, {12, 12, 179: 12, 645: 12}, {14, 14, 645: 14}, - {493: 2638, 722: 4993}, + {493: 2639, 722: 4994}, // 2545 {10, 10, 645: 10}, {18, 18}, {37: 55, 143: 55, 493: 55}, {59, 59}, - {493: 2638, 722: 5000}, + {493: 2639, 722: 5001}, // 2550 - {493: 2638, 722: 4999}, + {493: 2639, 722: 5000}, {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, 1122: 5006, 1291: 5004}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5006, 1123: 5007, 1291: 5005}, {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}, - {69, 69, 7: 5012}, - {658: 5008, 674: 5009, 1222: 5007}, + {69, 69, 7: 5013}, + {658: 5009, 674: 5010, 1222: 5008}, {61, 61, 7: 61}, {66, 66, 7: 66}, // 2560 - {65, 65, 7: 65, 136: 5011}, - {63, 63, 7: 63, 136: 5010}, + {65, 65, 7: 65, 136: 5012}, + {63, 63, 7: 63, 136: 5011}, {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, 1122: 5013}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5006, 1123: 5014}, // 2565 {60, 60, 7: 60}, {70, 70}, - {136: 4979, 860: 96, 1054: 5018}, - {461: 5017}, + {136: 4980, 862: 96, 1055: 5019}, + {461: 5018}, {54, 54}, // 2570 - {860: 5019}, - {461: 5020}, - {474: 5021, 481: 2052, 492: 5022, 1018: 5023}, + {862: 5020}, + {461: 5021}, + {474: 5022, 481: 2053, 491: 5023, 1019: 5024}, + {2052, 2052, 458: 2052, 2052, 2052, 465: 2052, 481: 2052, 484: 2052, 553: 2052, 632: 2052}, {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, 1161: 5027}, - {94, 94, 98: 3828, 107: 3827, 459: 94, 474: 94, 498: 94, 645: 94, 820: 3826, 1028: 5031}, + {481: 5025}, + {553: 5026}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5027}, + {98, 98, 98: 98, 107: 98, 459: 98, 474: 98, 498: 98, 634: 5029, 645: 98, 1161: 5028}, + {94, 94, 98: 3829, 107: 3828, 459: 94, 474: 94, 498: 94, 645: 94, 820: 3827, 1029: 5032}, // 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, 793: 5030}, + {498: 5030}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 529: 3646, 648: 3378, 2665, 2666, 2664, 725: 3645, 794: 5031}, {97, 97, 98: 97, 107: 97, 459: 97, 474: 97, 498: 97, 645: 97}, - {81, 81, 459: 81, 474: 81, 498: 81, 645: 3849, 928: 5032}, - {100, 100, 459: 100, 474: 5034, 498: 100, 1203: 5033}, + {81, 81, 459: 81, 474: 81, 498: 81, 645: 3850, 928: 5033}, + {100, 100, 459: 100, 474: 5035, 498: 100, 1203: 5034}, // 2585 - {2238, 2238, 459: 5037, 498: 2238, 1167: 5038}, - {493: 2638, 722: 5035}, - {645: 5036}, + {2239, 2239, 459: 5038, 498: 2239, 1167: 5039}, + {493: 2639, 722: 5036}, + {645: 5037}, {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, 983: 5049, 1165: 5048, 5052}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 2245, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 554: 3238, 648: 4025, 2665, 2666, 2664, 697: 5052, 729: 5051, 984: 5050, 1165: 5049, 5053}, // 2590 - {75, 75, 498: 5040, 1220: 5039}, + {75, 75, 498: 5041, 1220: 5040}, {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, 1052: 5042, 1219: 5041}, - {74, 74, 7: 5046}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3665, 2665, 2666, 2664, 698: 5044, 1053: 5043, 1219: 5042}, + {74, 74, 7: 5047}, {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, 790: 5045}, + {482: 5045}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3788, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3784, 784: 5046}, {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, 1052: 5047}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3665, 2665, 2666, 2664, 698: 5044, 1053: 5048}, {73, 73, 7: 73}, // 2600 - {7: 5054, 57: 2243}, - {7: 2242, 57: 2242}, + {7: 5055, 57: 2244}, + {7: 2243, 57: 2243}, + {7: 2241, 57: 2241}, {7: 2240, 57: 2240}, - {7: 2239, 57: 2239}, - {57: 5053}, + {57: 5054}, // 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, 983: 5055}, - {7: 2241, 57: 2241}, - {7: 158, 161: 158, 458: 158, 487: 158, 554: 1779, 637: 158, 654: 1779}, + {2238, 2238, 498: 2238}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 554: 3238, 648: 4025, 2665, 2666, 2664, 697: 5052, 729: 5051, 984: 5056}, + {7: 2242, 57: 2242}, + {7: 158, 163: 158, 458: 158, 487: 158, 554: 1779, 637: 158, 654: 1779}, {7: 123, 458: 123, 123, 487: 123, 554: 1750, 637: 123, 654: 1750}, // 2610 {7: 137, 458: 137, 137, 487: 137, 554: 1724, 637: 137, 654: 1724}, @@ -9141,39 +9142,39 @@ var ( {7: 133, 458: 133, 133, 487: 133, 554: 1611, 637: 133, 654: 1611}, {7: 138, 458: 138, 138, 487: 138, 554: 1604, 637: 138, 654: 1604}, // 2615 - {305: 5165, 370: 5164, 554: 1586, 654: 1586}, + {305: 5166, 370: 5165, 554: 1586, 654: 1586}, {7: 125, 458: 125, 125, 487: 125, 554: 1583, 637: 125, 654: 1583}, {7: 114, 458: 114, 114, 487: 114, 554: 1580, 637: 114, 654: 1580}, - {554: 5162, 654: 5161}, + {554: 5163, 654: 5162}, {7: 701, 458: 701, 487: 701, 554: 248, 637: 701, 654: 248}, // 2620 {7: 700, 458: 700, 487: 700, 637: 700}, - {7: 154, 161: 5160, 458: 154, 487: 154, 637: 154}, + {7: 154, 163: 5161, 458: 154, 487: 154, 637: 154}, {7: 156, 458: 156, 487: 156, 637: 156}, {7: 155, 458: 155, 487: 155, 637: 155}, - {487: 5158}, + {487: 5159}, // 2625 - {7: 134, 458: 134, 134, 481: 5156, 487: 134, 637: 134}, + {7: 134, 458: 134, 134, 481: 5157, 487: 134, 637: 134}, {7: 151, 458: 151, 487: 151, 637: 151}, - {7: 5108, 458: 5109, 487: 5110}, - {7: 149, 458: 149, 5105, 487: 149, 637: 149}, - {7: 147, 180: 5104, 458: 147, 147, 487: 147, 637: 147}, + {7: 5109, 458: 5110, 487: 5111}, + {7: 149, 458: 149, 5106, 487: 149, 637: 149}, + {7: 147, 180: 5105, 458: 147, 147, 487: 147, 637: 147}, // 2630 - {7: 145, 266: 5103, 458: 145, 145, 487: 145, 637: 145}, - {7: 144, 30: 5097, 99: 5099, 156: 5098, 158: 5096, 164: 5100, 266: 5101, 458: 144, 144, 487: 144, 637: 144}, + {7: 145, 266: 5104, 458: 145, 145, 487: 145, 637: 145}, + {7: 144, 30: 5098, 99: 5100, 158: 5099, 160: 5097, 166: 5101, 266: 5102, 458: 144, 144, 487: 144, 637: 144}, {7: 141, 458: 141, 141, 487: 141, 637: 141}, {7: 140, 458: 140, 140, 487: 140, 637: 140}, - {7: 139, 164: 5095, 458: 139, 139, 487: 139, 637: 139}, + {7: 139, 166: 5096, 458: 139, 139, 487: 139, 637: 139}, // 2635 {7: 136, 458: 136, 136, 487: 136, 637: 136}, {7: 135, 458: 135, 135, 487: 135, 637: 135}, - {99: 5094, 1001: 5093}, + {99: 5095, 1002: 5094}, {7: 131, 458: 131, 131, 487: 131, 637: 131}, - {889: 5092}, + {891: 5093}, // 2640 {7: 129, 458: 129, 129, 487: 129, 637: 129}, {7: 126, 458: 126, 126, 487: 126, 637: 126}, - {110: 5091}, + {110: 5092}, {7: 121, 458: 121, 121, 487: 121, 637: 121}, {7: 130, 458: 130, 130, 487: 130, 637: 130}, // 2645 @@ -9183,7 +9184,7 @@ var ( {7: 143, 458: 143, 143, 487: 143, 637: 143}, {7: 142, 458: 142, 142, 487: 142, 637: 142}, // 2650 - {110: 5102}, + {110: 5103}, {7: 120, 458: 120, 120, 487: 120, 637: 120}, {7: 118, 458: 118, 118, 487: 118, 637: 118}, {7: 116, 458: 116, 116, 487: 116, 637: 116}, @@ -9191,50 +9192,50 @@ 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, 794: 5106}, - {7: 4027, 57: 5107}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4026, 795: 5107}, + {7: 4028, 57: 5108}, {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, 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, 1070: 5117}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 5057, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 5059, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 5065, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 5061, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 5058, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 5066, 3096, 2832, 3051, 5060, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 5063, 2745, 2746, 2982, 5064, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 5062, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 5068, 483: 5091, 555: 5085, 631: 5089, 5074, 635: 5084, 638: 5078, 641: 5087, 648: 3378, 2665, 2666, 2664, 653: 5079, 656: 5083, 661: 5080, 725: 5067, 731: 5082, 787: 5069, 804: 5073, 838: 5088, 846: 5086, 920: 5070, 938: 5071, 5077, 944: 5072, 5156, 954: 5081, 956: 5090}, + {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, 5123, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 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: 5122, 940: 5124, 1062: 5125}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5114, 848: 5115}, + {714, 714, 7: 714, 14: 714, 58: 714, 99: 714, 141: 714, 460: 714, 468: 714, 482: 714, 554: 5120, 637: 714, 652: 714, 654: 5119, 714}, + {1168, 1168, 7: 1168, 14: 1168, 58: 1168, 99: 1168, 141: 1168, 459: 3655, 1168, 468: 1168, 482: 1168, 637: 1168, 652: 1168, 655: 1168, 1071: 5118}, // 2665 {710, 710, 7: 710, 460: 710}, - {102, 102, 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, 461: 3376, 556: 5112, 648: 3377, 2664, 2665, 2663, 725: 5111, 760: 5116}, + {102, 102, 7: 5116}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5117}, {709, 709, 7: 709, 460: 709}, {711, 711, 7: 711, 14: 711, 58: 711, 99: 711, 141: 711, 460: 711, 468: 711, 482: 711, 637: 711, 652: 711, 655: 711}, // 2670 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5120}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 648: 3378, 2665, 2666, 2664, 725: 5121}, {712, 712, 7: 712, 14: 712, 58: 712, 99: 712, 141: 712, 460: 712, 468: 712, 482: 712, 637: 712, 652: 712, 655: 712}, {713, 713, 7: 713, 14: 713, 58: 713, 99: 713, 141: 713, 460: 713, 468: 713, 482: 713, 637: 713, 652: 713, 655: 713}, {2: 111, 111, 111, 111, 111, 8: 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 58: 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 509: 111}, {2: 110, 110, 110, 110, 110, 8: 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 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, 1084: 5127}, - {487: 108, 637: 108, 639: 5153}, - {487: 104, 637: 104, 639: 5150}, - {487: 5128}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 509: 5126, 648: 5127, 2665, 2666, 2664, 1085: 5128}, + {487: 108, 637: 108, 639: 5154}, + {487: 104, 637: 104, 639: 5151}, + {487: 5129}, // 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, 870: 5130, 906: 5131}, - {190, 190, 7: 190, 14: 190, 58: 190, 141: 5135, 460: 190, 652: 190, 1153: 5134}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5130, 875: 5131, 906: 5132}, + {190, 190, 7: 190, 14: 190, 58: 190, 141: 5136, 460: 190, 652: 190, 1153: 5135}, {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, 870: 5133}, + {103, 103, 7: 5133}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5130, 875: 5134}, // 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, 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}, + {460: 5138, 646: 5137}, + {14: 5149, 461: 5146, 877: 5148}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 648: 3378, 2665, 2666, 2664, 725: 5140, 1154: 5139}, // 2690 - {188, 188, 7: 188, 14: 188, 58: 188, 460: 188, 465: 5141, 646: 5140, 652: 188}, + {188, 188, 7: 188, 14: 188, 58: 188, 460: 188, 465: 5142, 646: 5141, 652: 188}, {184, 184, 7: 184, 14: 184, 58: 184, 460: 184, 465: 184, 646: 184, 652: 184}, - {461: 5145, 872: 5146}, - {461: 5143, 562: 5144, 1037: 5142}, + {461: 5146, 877: 5147}, + {461: 5144, 562: 5145, 1038: 5143}, {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,50 +9244,50 @@ 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, 1037: 5149}, + {461: 5144, 562: 5145, 1038: 5150}, {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 509: 5152, 648: 5153, 2665, 2666, 2664}, {487: 106, 637: 106}, {487: 105, 637: 105}, // 2705 - {509: 5154}, + {509: 5155}, {487: 107, 637: 107}, {7: 150, 458: 150, 487: 150, 637: 150}, - {267: 5157}, + {267: 5158}, {7: 152, 458: 152, 487: 152, 637: 152}, // 2710 - {267: 5159}, + {267: 5160}, {7: 153, 458: 153, 487: 153, 637: 153}, - {7: 157, 161: 157, 458: 157, 487: 157, 637: 157}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5163}, + {7: 157, 163: 157, 458: 157, 487: 157, 637: 157}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 648: 3378, 2665, 2666, 2664, 725: 5164}, {702, 702, 7: 702, 458: 702, 487: 702, 637: 702}, // 2715 {703, 703, 7: 703, 458: 703, 487: 703, 637: 703}, {7: 128, 458: 128, 128, 487: 128, 637: 128}, {7: 127, 458: 127, 127, 487: 127, 637: 127}, - {458: 5206, 554: 1697, 654: 1697}, - {7: 5108, 458: 5168, 637: 5169}, + {458: 5207, 554: 1697, 654: 1697}, + {7: 5109, 458: 5169, 637: 5170}, // 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, 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, 1084: 5172}, - {637: 5173}, + {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, 5123, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 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: 5122, 940: 5124, 1062: 5172}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5114, 848: 5171}, + {165, 165, 7: 5116}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 509: 5126, 648: 5127, 2665, 2666, 2664, 1085: 5173}, + {637: 5174}, // 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, 870: 5130, 906: 5174}, - {215, 215, 7: 5132, 460: 215, 652: 5176, 941: 5175, 5177}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5130, 875: 5131, 906: 5175}, + {215, 215, 7: 5133, 460: 215, 652: 5177, 941: 5176, 5178}, {214, 214, 14: 214, 58: 214, 460: 214}, - {131: 5197, 133: 5195, 5198, 5196, 346: 5190, 393: 5192, 943: 5194, 1258: 5193, 1276: 5191}, - {164, 164, 460: 5179, 1140: 5178}, + {131: 5198, 133: 5196, 5199, 5197, 346: 5191, 393: 5193, 943: 5195, 1258: 5194, 1276: 5192}, + {164, 164, 460: 5180, 1140: 5179}, // 2730 {167, 167}, - {126: 5183, 5181, 5182, 5184, 837: 5180}, - {889: 5189}, - {493: 2638, 722: 5188}, - {493: 2638, 722: 5187}, + {126: 5184, 5182, 5183, 5185, 838: 5181}, + {891: 5190}, + {493: 2639, 722: 5189}, + {493: 2639, 722: 5188}, // 2735 - {493: 2638, 722: 5186}, - {493: 2638, 722: 5185}, + {493: 2639, 722: 5187}, + {493: 2639, 722: 5186}, {159, 159}, {160, 160}, {161, 161}, @@ -9297,208 +9298,208 @@ 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, 943: 5204}, - {209, 209, 14: 209, 58: 209, 131: 209, 133: 209, 209, 209, 460: 209, 491: 209}, + {210, 210, 14: 210, 58: 210, 131: 5198, 133: 5196, 5199, 5197, 460: 210, 492: 5204, 943: 5205}, + {209, 209, 14: 209, 58: 209, 131: 209, 133: 209, 209, 209, 460: 209, 492: 209}, + {461: 5203}, {461: 5202}, {461: 5201}, - {461: 5200}, // 2750 - {461: 5199}, - {203, 203, 14: 203, 58: 203, 131: 203, 133: 203, 203, 203, 460: 203, 491: 203}, - {204, 204, 14: 204, 58: 204, 131: 204, 133: 204, 204, 204, 460: 204, 491: 204}, - {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}, + {461: 5200}, + {203, 203, 14: 203, 58: 203, 131: 203, 133: 203, 203, 203, 460: 203, 492: 203}, + {204, 204, 14: 204, 58: 204, 131: 204, 133: 204, 204, 204, 460: 204, 492: 204}, + {205, 205, 14: 205, 58: 205, 131: 205, 133: 205, 205, 205, 460: 205, 492: 205}, + {206, 206, 14: 206, 58: 206, 131: 206, 133: 206, 206, 206, 460: 206, 492: 206}, // 2755 - {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}, + {131: 5198, 133: 5196, 5199, 5197, 943: 5206}, + {207, 207, 14: 207, 58: 207, 131: 207, 133: 207, 207, 207, 460: 207, 492: 207}, + {208, 208, 14: 208, 58: 208, 131: 208, 133: 208, 208, 208, 460: 208, 492: 208}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5208}, + {637: 5209}, // 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, 1140: 5210}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5114, 848: 5210}, + {164, 164, 7: 5116, 460: 5180, 1140: 5211}, {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}, + {2123, 2123, 7: 2123, 13: 2123, 15: 2123, 2123, 2123, 2123, 2123, 2123, 2123, 2123, 2123, 2123, 2123, 2123, 28: 2123, 464: 2123, 467: 2123, 486: 2123, 2123, 489: 2123, 507: 2123, 634: 2123, 637: 2123}, {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: 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, 491: 819, 493: 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, 746: 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, 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}, + {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: 5219, 5218, 5217, 824: 5220, 869: 5221}, + {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, 491: 1023, 493: 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, 746: 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, 491: 1022, 493: 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, 746: 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: 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, 491: 1021, 493: 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, 746: 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, 783: 5227, 786: 5226, 5224, 834: 5222, 869: 5223}, + {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: 4072, 565: 1803, 839: 5222}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 5231, 565: 5226, 648: 3806, 2665, 2666, 2664, 696: 5230, 724: 5229, 785: 5228, 788: 5227, 5225, 835: 5223, 874: 5224}, {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}, - {7: 894, 469: 5243, 5244, 498: 5333, 500: 5242, 503: 5245, 5241, 5246, 5247, 803: 5240, 809: 5239}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5330, 2664, 2665, 2663}, + {7: 5277, 498: 5347}, + {7: 894, 469: 5244, 5245, 498: 5334, 500: 5243, 503: 5246, 5242, 5247, 5248, 803: 5241, 810: 5240}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5331, 2665, 2666, 2664}, {892, 892, 7: 892, 57: 892, 458: 892, 460: 892, 466: 892, 468: 892, 892, 892, 476: 892, 892, 479: 892, 892, 892, 483: 892, 488: 892, 892, 892, 498: 892, 892, 892, 892, 892, 892, 892, 892, 892, 508: 892}, {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, 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}, + {887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 887, 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, 5281, 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, 843: 5280}, + {885, 885, 2901, 2749, 2785, 2903, 2676, 885, 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 885, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 885, 460: 885, 465: 5238, 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: 5237, 2665, 2666, 2664, 900: 5236, 5235}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 5231, 2497, 484: 2496, 553: 2495, 565: 5226, 632: 2491, 648: 3806, 2665, 2666, 2664, 696: 5234, 724: 5229, 738: 3766, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 3768, 3767, 3765, 785: 5228, 788: 5227, 5233, 835: 5223, 874: 5232}, + {7: 5277, 57: 5278}, + {894, 894, 7: 894, 57: 894, 458: 894, 460: 894, 466: 894, 468: 894, 5244, 5245, 476: 894, 894, 479: 894, 894, 894, 483: 894, 488: 894, 894, 894, 498: 894, 894, 5243, 894, 894, 5246, 5242, 5247, 5248, 803: 5241, 810: 5240}, // 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, 900: 5235, 5234}, + {2: 2901, 2749, 2785, 2903, 2676, 885, 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 885, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 465: 5238, 780, 469: 885, 885, 476: 780, 780, 480: 2631, 488: 2632, 490: 2628, 500: 885, 503: 885, 885, 885, 885, 648: 5237, 2665, 2666, 2664, 753: 3776, 3777, 900: 5236, 5235}, {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5239, 2665, 2666, 2664}, // 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, 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, 783: 5227, 786: 5226, 5250}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 5231, 648: 3806, 2665, 2666, 2664, 696: 5230, 724: 5229, 785: 5228, 788: 5227, 5270}, + {503: 853, 893: 5257, 1075: 5261}, + {469: 5244, 5245, 503: 5254, 803: 5255}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 5231, 648: 3806, 2665, 2666, 2664, 696: 5230, 724: 5229, 785: 5228, 788: 5227, 5251}, // 2795 - {503: 855, 891: 855}, - {503: 854, 891: 854}, + {503: 855, 893: 855}, + {503: 854, 893: 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: 5250}, {503: 5249}, - {503: 5248}, // 2800 {2: 849, 849, 849, 849, 849, 8: 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 58: 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 849, 459: 849}, {2: 850, 850, 850, 850, 850, 8: 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 58: 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, 459: 850}, - {857, 857, 7: 857, 57: 857, 458: 5251, 460: 857, 466: 857, 468: 857, 857, 857, 476: 857, 857, 479: 857, 857, 857, 483: 857, 488: 857, 857, 857, 498: 857, 857, 857, 857, 857, 857, 857, 857, 857, 508: 857, 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, 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}, + {857, 857, 7: 857, 57: 857, 458: 5252, 460: 857, 466: 857, 468: 857, 857, 857, 476: 857, 857, 479: 857, 857, 857, 483: 857, 488: 857, 857, 857, 498: 857, 857, 857, 857, 857, 857, 857, 857, 857, 508: 857, 803: 5241, 810: 5240}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 5253}, + {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, 492: 3235, 494: 3233, 3234, 3232, 3230, 856, 856, 856, 856, 856, 856, 856, 856, 856, 508: 856, 720: 3231, 3229}, // 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, 783: 5227, 786: 5226, 5259}, - {503: 853, 891: 5256, 1074: 5255}, - {503: 5257}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 5231, 648: 3806, 2665, 2666, 2664, 696: 5230, 724: 5229, 785: 5228, 788: 5227, 5260}, + {503: 853, 893: 5257, 1075: 5256}, + {503: 5258}, {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, 783: 5227, 786: 5226, 5258}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 5231, 648: 3806, 2665, 2666, 2664, 696: 5230, 724: 5229, 785: 5228, 788: 5227, 5259}, // 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, 783: 5227, 786: 5226, 5262}, - {458: 5263, 468: 5264, 5243, 5244, 500: 5242, 503: 5245, 5241, 5246, 5247, 803: 5240, 809: 5239}, + {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: 5241, 810: 5240}, + {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: 5241, 810: 5240}, + {503: 5262}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 5231, 648: 3806, 2665, 2666, 2664, 696: 5230, 724: 5229, 785: 5228, 788: 5227, 5263}, + {458: 5264, 468: 5265, 5244, 5245, 500: 5243, 503: 5246, 5242, 5247, 5248, 803: 5241, 810: 5240}, // 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, 794: 5266}, - {7: 4027, 57: 5267}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 5269}, + {459: 5266}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4026, 795: 5267}, + {7: 4028, 57: 5268}, {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 - {861, 861, 7: 861, 57: 861, 458: 861, 460: 861, 466: 861, 468: 861, 861, 861, 476: 861, 861, 479: 861, 861, 861, 483: 861, 488: 861, 861, 861, 3234, 494: 3232, 3233, 3231, 3229, 861, 861, 861, 861, 861, 861, 861, 861, 861, 508: 861, 720: 3230, 3228}, - {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, 794: 5273}, + {861, 861, 7: 861, 57: 861, 458: 861, 460: 861, 466: 861, 468: 861, 861, 861, 476: 861, 861, 479: 861, 861, 861, 483: 861, 488: 861, 861, 861, 492: 3235, 494: 3233, 3234, 3232, 3230, 861, 861, 861, 861, 861, 861, 861, 861, 861, 508: 861, 720: 3231, 3229}, + {864, 864, 7: 864, 57: 864, 458: 5271, 460: 864, 466: 864, 468: 5272, 5244, 5245, 476: 864, 864, 479: 864, 864, 864, 483: 864, 488: 864, 864, 864, 498: 864, 864, 5243, 864, 864, 5246, 5242, 5247, 5248, 508: 864, 803: 5241, 810: 5240}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 5276}, + {459: 5273}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4026, 795: 5274}, // 2825 - {7: 4027, 57: 5274}, + {7: 4028, 57: 5275}, {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, 783: 5227, 786: 5226, 5232, 834: 5278}, + {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, 492: 3235, 494: 3233, 3234, 3232, 3230, 863, 863, 863, 863, 863, 863, 863, 863, 863, 508: 863, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 5231, 565: 5226, 648: 3806, 2665, 2666, 2664, 696: 5230, 724: 5229, 785: 5228, 788: 5227, 5233, 835: 5279}, {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, 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, 774: 5282}, - {7: 4626, 57: 5283}, + {885, 885, 2901, 2749, 2785, 2903, 2676, 885, 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 885, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 885, 460: 885, 465: 5238, 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: 5237, 2665, 2666, 2664, 660: 885, 900: 5236, 5285}, + {459: 5282}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4626, 2665, 2666, 2664, 776: 5283}, + {7: 4627, 57: 5284}, // 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, 908: 5285, 1151: 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: 5287, 908: 5286, 1151: 5288}, {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, 924: 5293, 5292, 1040: 5294, 5288}, + {213: 5329}, + {866, 866, 7: 866, 57: 866, 458: 866, 460: 866, 466: 866, 468: 866, 866, 866, 474: 5291, 476: 866, 866, 479: 866, 866, 866, 483: 866, 485: 5292, 488: 866, 866, 866, 498: 866, 866, 866, 866, 866, 866, 866, 866, 866, 508: 866, 511: 5290, 866, 924: 5294, 5293, 1041: 5295, 5289}, // 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}, - {563: 4299, 635: 4300, 813: 5307}, - {563: 4299, 635: 4300, 813: 5306}, - {459: 878, 479: 5296, 1205: 5297}, + {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: 5310, 1294: 5311}, + {563: 4300, 635: 4301, 814: 5309}, + {563: 4300, 635: 4301, 814: 5308}, + {563: 4300, 635: 4301, 814: 5307}, + {459: 878, 479: 5297, 1205: 5298}, // 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, 924: 5295, 5292}, + {865, 865, 7: 865, 57: 865, 458: 865, 460: 865, 466: 865, 468: 865, 865, 865, 474: 5291, 476: 865, 865, 479: 865, 865, 865, 483: 865, 485: 5292, 488: 865, 865, 865, 498: 865, 865, 865, 865, 865, 865, 865, 865, 865, 508: 865, 511: 5290, 865, 924: 5296, 5293}, {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}, + {490: 5303, 499: 5304, 503: 5302}, + {459: 5299}, // 2850 - {2: 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, 873, 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 3024, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 573: 4905, 648: 4904, 2664, 2665, 2663, 840: 5299}, - {7: 4907, 57: 5300}, + {2: 2901, 2749, 2785, 2903, 2676, 873, 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 873, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 573: 4906, 648: 4905, 2665, 2666, 2664, 841: 5300}, + {7: 4908, 57: 5301}, {874, 874, 7: 874, 57: 874, 458: 874, 460: 874, 466: 874, 468: 874, 874, 874, 474: 874, 476: 874, 874, 479: 874, 874, 874, 483: 874, 485: 874, 488: 874, 874, 874, 498: 874, 874, 874, 874, 874, 874, 874, 874, 874, 508: 874, 511: 874, 874}, {459: 877}, - {646: 5305}, + {646: 5306}, // 2855 - {646: 5304}, + {646: 5305}, {459: 875}, {459: 876}, {459: 879, 479: 879}, {459: 880, 479: 880}, // 2860 {459: 881, 479: 881}, - {15: 5314, 295: 5313, 376: 5312, 459: 978, 1293: 5311}, + {15: 5315, 295: 5314, 376: 5313, 459: 978, 1293: 5312}, {890, 890, 7: 890, 57: 890, 458: 890, 460: 890, 466: 890, 468: 890, 890, 890, 476: 890, 890, 479: 890, 890, 890, 483: 890, 488: 890, 890, 890, 498: 890, 890, 890, 890, 890, 890, 890, 890, 890, 508: 890}, - {459: 5315}, + {459: 5316}, {459: 977}, // 2865 {459: 976}, {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, 1092: 5318}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 5318, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 5317}, + {57: 974, 355: 5326, 492: 3235, 494: 3233, 3234, 3232, 3230, 510: 5325, 720: 3231, 3229, 1295: 5324}, + {971, 971, 7: 971, 57: 971, 208: 5320, 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, 1093: 5319}, // 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}, - {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, 5321}, - {57: 5322, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, + {459: 5321}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 5322}, + {57: 5323, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, {970, 970, 7: 970, 57: 970, 458: 970, 460: 970, 466: 970, 468: 970, 970, 970, 476: 970, 970, 479: 970, 970, 970, 483: 970, 488: 970, 970, 970, 498: 970, 970, 970, 970, 970, 970, 970, 970, 970, 508: 970}, // 2875 - {57: 5326}, + {57: 5327}, {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, 1092: 5327}, + {971, 971, 7: 971, 57: 971, 208: 5320, 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, 1093: 5328}, {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, 783: 5227, 786: 5226, 5331}, - {469: 5243, 5244, 500: 5242, 503: 5245, 5241, 5246, 5247, 508: 5332, 803: 5240, 809: 5239}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 5330}, + {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, 492: 3235, 494: 3233, 3234, 3232, 3230, 1808, 1808, 1808, 1808, 1808, 1808, 1808, 1808, 1808, 508: 1808, 511: 1808, 1808, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 5231, 648: 3806, 2665, 2666, 2664, 696: 5230, 724: 5229, 785: 5228, 788: 5227, 5332}, + {469: 5244, 5245, 500: 5243, 503: 5246, 5242, 5247, 5248, 508: 5333, 803: 5241, 810: 5240}, {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, 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, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 5335, 876: 5336, 909: 5337}, + {482: 5345}, + {2274, 2274, 7: 2274, 468: 2274, 480: 2274, 489: 2274, 2274}, + {237, 237, 7: 5338, 468: 237, 480: 237, 489: 2625, 237, 777: 2626, 5339}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 5335, 876: 5344}, // 2890 - {1258, 1258, 468: 1258, 480: 1258, 490: 2627, 753: 2628, 797: 5339}, - {848, 848, 468: 848, 480: 5340, 1049: 5341}, - {493: 2638, 564: 2640, 722: 2637, 730: 2639, 861: 5342}, + {1258, 1258, 468: 1258, 480: 1258, 490: 2628, 753: 2629, 797: 5340}, + {848, 848, 468: 848, 480: 5341, 1050: 5342}, + {493: 2639, 564: 2641, 722: 2638, 730: 2640, 863: 5343}, {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, 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, 871: 5335, 909: 5347}, - {237, 237, 7: 5337, 468: 237, 489: 2624, 777: 2625, 5348}, + {2273, 2273, 7: 2273, 468: 2273, 480: 2273, 489: 2273, 2273}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3788, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3784, 784: 5346}, + {2275, 2275, 7: 2275, 468: 2275, 480: 2275, 489: 2275, 2275}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 5335, 876: 5336, 909: 5348}, + {237, 237, 7: 5338, 468: 237, 489: 2625, 777: 2626, 5349}, // 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5352}, {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, 1124: 5361, 1193: 5355, 1223: 5360}, + {31: 5365, 110: 5355, 125: 5358, 142: 556, 180: 5357, 186: 5368, 195: 5366, 212: 5359, 223: 5363, 243: 5367, 246: 5360, 529: 5364, 553: 5354, 1125: 5362, 1193: 5356, 1223: 5361}, // 2905 + {1990, 1990, 1990, 1990, 1990, 1990, 1990, 8: 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 58: 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 460: 1990, 552: 1990}, {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}, {566, 566}, {563, 563}, {562, 562}, // 2910 - {203: 5374}, + {204: 5375}, {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, 782: 4192, 1123: 5368}, + {142: 5374}, + {547, 547, 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 460: 547, 648: 3806, 2665, 2666, 2664, 724: 3807, 782: 4193, 1124: 5369}, {557, 557}, // 2915 {142: 555}, @@ -9507,326 +9508,326 @@ var ( {142: 552}, {142: 551}, // 2920 - {543, 543, 460: 5370, 1322: 5369}, + {543, 543, 460: 5371, 1322: 5370}, {558, 558}, - {658: 5371}, - {483: 5372}, + {658: 5372}, + {483: 5373}, {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, 936: 5376}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5376, 2665, 2666, 2664, 936: 5377}, {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}, + {561, 561, 7: 5378}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5379, 2665, 2666, 2664}, // 2930 {564, 564, 7: 564}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 5477, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 5478, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 2849, 2780, 2891, 2811, 2814, 3096, 3071, 3072, 3073, 3074, 3097, 3068, 3069, 3070, 2830, 5479, 3282, 3283, 3090, 3076, 3077, 3078, 3108, 3266, 648: 3805, 2664, 2665, 2663, 724: 5480}, - {553: 5463, 635: 5464}, - {635: 5460}, - {553: 5455, 635: 5454}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 5478, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 5479, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 5480, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5481}, + {553: 5464, 635: 5465}, + {635: 5461}, + {553: 5456, 635: 5455}, // 2935 - {553: 5452}, - {312: 5446}, - {138: 5443, 209: 5445, 321: 5441, 350: 5442, 899: 5444}, - {191: 5438, 194: 5437}, - {553: 5396}, + {553: 5453}, + {312: 5447}, + {138: 5444, 210: 5446, 321: 5442, 350: 5443, 899: 5445}, + {191: 5439, 194: 5438}, + {553: 5397}, // 2940 + {138: 5396}, {138: 5395}, {138: 5394}, - {138: 5393}, - {378: 5392}, + {378: 5393}, {669, 669}, // 2945 {674, 674}, {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, 915: 5399}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5398}, + {661: 5399, 915: 5400}, // 2950 - {156: 5401, 160: 5402, 553: 2004, 931: 5400}, + {158: 5402, 162: 5403, 553: 2005, 931: 5401}, {677, 677}, - {553: 5404}, - {110: 2003, 553: 2003}, - {156: 5403}, + {553: 5405}, + {110: 2004, 553: 2004}, + {158: 5404}, // 2955 - {110: 2002, 553: 2002}, - {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, 779: 4143, 798: 4142, 821: 4144, 825: 4154, 828: 4169, 904: 5425, 998: 5424}, + {110: 2003, 553: 2003}, + {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: 4702, 769: 5406}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5407}, + {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, 5411, 425, 464: 425, 425, 425, 425, 474: 425, 425, 484: 425, 425, 425, 491: 425, 507: 5410, 553: 425, 632: 425, 634: 425, 425, 1217: 5409, 1290: 5408}, + {385, 385, 4: 4132, 4134, 389, 13: 2107, 4151, 4078, 4090, 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 4149, 4169, 4153, 4140, 4133, 4136, 4135, 4138, 4139, 4141, 4148, 389, 4159, 4160, 4146, 4147, 4152, 4154, 4166, 4165, 4171, 4167, 4164, 4157, 4162, 4163, 4156, 4158, 4161, 4150, 458: 385, 385, 385, 464: 4131, 385, 4168, 2107, 474: 385, 385, 484: 385, 4845, 2107, 491: 385, 553: 385, 632: 385, 634: 2107, 4137, 758: 4088, 762: 4089, 766: 4142, 779: 4144, 798: 4143, 821: 4145, 825: 4155, 829: 4170, 904: 5426, 999: 5425}, // 2960 - {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, 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}, + {2110, 2110, 458: 5419, 1065: 5418}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5417}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 507: 5412, 563: 2324, 571: 2324, 573: 2324, 629: 2324, 4479, 635: 2324, 648: 4025, 2665, 2666, 2664, 662: 2324, 2324, 729: 4346, 819: 4688, 832: 4820, 888: 4821, 949: 4822, 1122: 5413}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5415}, + {7: 4824, 57: 5414}, // 2965 - {424, 424, 4: 424, 424, 424, 13: 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 458: 424, 424, 424, 464: 424, 424, 424, 424, 474: 424, 424, 484: 424, 424, 424, 492: 424, 553: 424, 632: 424, 634: 424, 424}, - {57: 5415}, - {2037, 2037, 458: 2037}, + {424, 424, 4: 424, 424, 424, 13: 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 458: 424, 424, 424, 464: 424, 424, 424, 424, 474: 424, 424, 484: 424, 424, 424, 491: 424, 553: 424, 632: 424, 634: 424, 424}, + {57: 5416}, {2038, 2038, 458: 2038}, - {2110, 2110}, + {2039, 2039, 458: 2039}, + {2111, 2111}, // 2970 - {159: 5419}, - {357: 5421, 731: 5420}, + {152: 5420}, + {357: 5422, 731: 5421}, + {510: 5424}, {510: 5423}, - {510: 5422}, - {2107, 2107}, - // 2975 {2108, 2108}, - {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}, + // 2975 + {2109, 2109}, + {2105, 2105, 458: 2105, 2105, 2105, 465: 2105, 474: 2105, 5428, 484: 2105, 491: 2105, 553: 2105, 632: 2105, 1078: 5427}, + {384, 384, 4: 4132, 4134, 389, 4847, 13: 2107, 4151, 4078, 4090, 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 4149, 4169, 4153, 4140, 4133, 4136, 4135, 4138, 4139, 4141, 4148, 389, 4159, 4160, 4146, 4147, 4152, 4154, 4166, 4165, 4171, 4167, 4164, 4157, 4162, 4163, 4156, 4158, 4161, 4150, 458: 384, 384, 384, 464: 4131, 384, 4168, 2107, 474: 384, 384, 484: 384, 4845, 2107, 491: 384, 553: 384, 632: 384, 634: 2107, 4137, 758: 4088, 762: 4089, 766: 4142, 779: 4144, 798: 4143, 821: 4145, 825: 4155, 829: 4846}, + {2053, 2053, 458: 2053, 2053, 2053, 465: 2053, 474: 5022, 484: 2053, 491: 5023, 553: 2053, 632: 2053, 1019: 5429}, + {646: 4917}, // 2980 - {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, 1064: 5436}, - {2046, 2046, 458: 2046}, + {2050, 2050, 458: 2050, 2050, 2050, 465: 5431, 484: 2050, 553: 2050, 632: 2050, 1152: 5430}, + {2048, 2048, 458: 2048, 2498, 2497, 484: 2496, 553: 2495, 632: 2491, 696: 5436, 738: 5434, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 5435, 5433, 3765, 1173: 5432}, + {2049, 2049, 458: 2049, 2049, 2049, 484: 2049, 553: 2049, 632: 2049}, + {2110, 2110, 458: 5419, 1065: 5437}, + {2047, 2047, 458: 2047}, // 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, 936: 5440}, + {2046, 2046, 458: 2046, 466: 781, 476: 781, 781}, + {2045, 2045, 458: 2045}, + {2044, 2044, 458: 2044, 466: 780, 476: 780, 780, 480: 2631, 488: 2632, 490: 2628, 753: 3776, 3777}, + {2112, 2112}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5376, 2665, 2666, 2664, 936: 5441}, // 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, 936: 5439}, - {679, 679, 7: 5377}, - {680, 680, 7: 5377}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5376, 2665, 2666, 2664, 936: 5440}, + {679, 679, 7: 5378}, + {680, 680, 7: 5378}, {682, 682}, {681, 681}, // 2995 {673, 673}, {672, 672}, {671, 671}, - {250: 5447}, - {493: 2638, 722: 3934, 748: 5449, 1060: 5448}, + {251: 5448}, + {493: 2639, 722: 3935, 748: 5450, 1061: 5449}, // 3000 - {685, 685, 7: 5450}, + {685, 685, 7: 5451}, {661, 661, 7: 661}, - {493: 2638, 722: 3934, 748: 5451}, + {493: 2639, 722: 3935, 748: 5452}, {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, 782: 5453}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 3807, 782: 5454}, // 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, 782: 5457}, - {678, 678, 7: 3808}, + {686, 686, 7: 3809}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5459}, + {483: 5457}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 3807, 782: 5458}, + {678, 678, 7: 3809}, // 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5460, 2665, 2666, 2664}, {688, 688}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5461}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5462}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5463, 2665, 2666, 2664}, {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, 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, 1036: 5468, 1198: 5467}, - {687, 687, 7: 5474}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 3807, 782: 5477}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5466}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5467, 2665, 2666, 2664}, + {690, 690, 459: 5470, 1037: 5469, 1198: 5468}, + {687, 687, 7: 5475}, // 3020 {664, 664, 7: 664}, - {493: 2638, 722: 3934, 748: 5470}, - {7: 5471}, - {493: 2638, 722: 3934, 748: 5472}, - {57: 5473}, + {493: 2639, 722: 3935, 748: 5471}, + {7: 5472}, + {493: 2639, 722: 3935, 748: 5473}, + {57: 5474}, // 3025 {662, 662, 7: 662}, - {459: 5469, 1036: 5475}, + {459: 5470, 1037: 5476}, {663, 663, 7: 663}, - {691, 691, 7: 3808}, - {153: 1591, 361: 5490, 384: 5491, 639: 1591, 1143: 5489}, + {691, 691, 7: 3809}, + {154: 1591, 361: 5491, 384: 5492, 639: 1591, 1143: 5490}, // 3030 - {695, 695, 153: 1433, 249: 5483, 5482, 639: 1433}, - {670, 670, 153: 1415, 639: 1415}, - {153: 5481}, + {695, 695, 154: 1433, 250: 5484, 5483, 639: 1433}, + {670, 670, 154: 1415, 639: 1415}, + {154: 5482}, {692, 692}, - {237, 237, 489: 2624, 493: 2638, 722: 3934, 748: 5487, 777: 2625, 5486}, + {237, 237, 489: 2625, 493: 2639, 722: 3935, 748: 5488, 777: 2626, 5487}, // 3035 - {360: 5484}, - {493: 2638, 722: 3934, 748: 5449, 1060: 5485}, - {684, 684, 7: 5450}, + {360: 5485}, + {493: 2639, 722: 3935, 748: 5450, 1061: 5486}, + {684, 684, 7: 5451}, {694, 694}, - {237, 237, 489: 2624, 777: 2625, 5488}, + {237, 237, 489: 2625, 777: 2626, 5489}, // 3040 {693, 693}, {683, 683}, - {493: 2638, 722: 5497}, - {332: 5493, 493: 2638, 638: 5494, 722: 5492}, + {493: 2639, 722: 5498}, + {332: 5494, 493: 2639, 638: 5495, 722: 5493}, {667, 667}, // 3045 - {493: 2638, 722: 5496}, - {493: 2638, 722: 5495}, + {493: 2639, 722: 5497}, + {493: 2639, 722: 5496}, {665, 665}, {666, 666}, {668, 668}, // 3050 {2: 259, 259, 259, 259, 259, 8: 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 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: 1752, 529: 259, 639: 1752, 647: 1752}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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, 793: 5591}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 5603, 482: 1750, 639: 1750, 647: 1750, 5602, 2665, 2666, 2664}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 5600, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 482: 1713, 639: 1713, 647: 1713, 5510, 2665, 2666, 2664, 817: 5553}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 482: 1707, 639: 1707, 647: 1707, 5510, 2665, 2666, 2664, 817: 5597}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 464: 5593, 482: 1705, 529: 3646, 639: 1705, 647: 1705, 3378, 2665, 2666, 2664, 725: 3645, 794: 5592}, // 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, 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, 905: 5556, 952: 5555}, - {773, 773, 7: 5545}, + {479: 5582, 482: 5581, 639: 1700, 647: 1700}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 5533, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 5534, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 5538, 464: 5578, 482: 1691, 638: 5576, 1691, 647: 1691, 3378, 2665, 2666, 2664, 725: 5067, 787: 5540, 807: 5541, 5539, 845: 5537, 1103: 5577, 1266: 5575}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 5573, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 482: 1689, 639: 1689, 647: 1689, 5510, 2665, 2666, 2664, 817: 5550}, + {173: 5558, 482: 1672, 639: 1672, 647: 1672, 658: 5559, 905: 5557, 953: 5556}, + {773, 773, 7: 5546}, // 3060 - {164: 5531}, - {482: 742, 639: 5529, 647: 742}, - {482: 5518, 647: 5519, 810: 5527}, - {482: 5518, 647: 5519, 810: 5522}, - {482: 5518, 647: 5519, 810: 5520}, + {166: 5532}, + {482: 742, 639: 5530, 647: 742}, + {482: 5519, 647: 5520, 811: 5528}, + {482: 5519, 647: 5520, 811: 5523}, + {482: 5519, 647: 5520, 811: 5521}, // 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, 793: 5516, 1160: 5515}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 464: 5518, 529: 3646, 648: 3378, 2665, 2666, 2664, 725: 3645, 794: 5517, 1160: 5516}, {720, 720, 7: 720}, {727, 727, 7: 727}, {726, 726, 7: 726}, {725, 725, 7: 725}, // 3070 - {2: 744, 744, 744, 744, 744, 8: 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 58: 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 461: 744, 744, 744, 744, 469: 744, 744, 744, 744, 744, 478: 744, 484: 744, 486: 744, 492: 744, 744, 529: 744, 552: 744, 554: 744, 744, 744, 744, 744, 744, 744, 744, 744, 564: 744, 744, 744, 744, 744, 744, 572: 744, 574: 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 636: 744}, - {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, 790: 5525, 845: 5526}, + {2: 744, 744, 744, 744, 744, 8: 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 58: 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 461: 744, 744, 744, 744, 469: 744, 744, 744, 744, 744, 478: 744, 484: 744, 486: 744, 491: 744, 493: 744, 529: 744, 552: 744, 554: 744, 744, 744, 744, 744, 744, 744, 744, 744, 564: 744, 744, 744, 744, 744, 744, 572: 744, 574: 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 744, 636: 744}, + {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, 491: 743, 493: 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: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 5522}, + {732, 732, 7: 732, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 5525, 3148, 461: 3128, 3146, 2658, 3788, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 5524, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3784, 784: 5526, 827: 5527}, // 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}, + {746, 746, 2901, 2749, 2785, 2903, 2676, 746, 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3751, 3141, 3224, 3140, 3137}, {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, 790: 5525, 845: 5528}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 5525, 3148, 461: 3128, 3146, 2658, 3788, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 5524, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3784, 784: 5526, 827: 5529}, // 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5531, 2665, 2666, 2664}, {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, 785: 5539, 806: 5540, 5538, 844: 5536, 1102: 5534}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 5533, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 5534, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 5538, 638: 5536, 648: 3378, 2665, 2666, 2664, 725: 5067, 787: 5540, 807: 5541, 5539, 845: 5537, 1103: 5535}, {704, 704, 7: 704, 554: 1779, 637: 704, 654: 1779}, // 3085 {763, 763, 554: 1613, 637: 763, 654: 1613}, - {637: 5543}, + {637: 5544}, {637: 762}, - {761, 761, 7: 5541, 637: 761}, + {761, 761, 7: 5542, 637: 761}, {705, 705, 7: 705, 554: 248, 637: 705, 654: 248}, // 3090 {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, 785: 5539, 806: 5542, 5538}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 5533, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 5538, 648: 3378, 2665, 2666, 2664, 725: 5067, 787: 5540, 807: 5543, 5539}, {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, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5114, 848: 5545}, + {764, 764, 7: 5116}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 5499, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 5502, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 5547, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 5548, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 5503, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 486: 3722, 554: 5513, 574: 5512, 634: 3720, 648: 5510, 2665, 2666, 2664, 759: 5514, 817: 5511, 960: 5549}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 482: 1713, 639: 1713, 647: 1713, 5510, 2665, 2666, 2664, 817: 5553}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 482: 1689, 639: 1689, 647: 1689, 5510, 2665, 2666, 2664, 817: 5550}, // 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, 790: 5525, 845: 5551}, + {482: 5519, 647: 5520, 811: 5551}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 5525, 3148, 461: 3128, 3146, 2658, 3788, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 5524, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3784, 784: 5526, 827: 5552}, {735, 735, 7: 735}, - {482: 5518, 647: 5519, 810: 5553}, + {482: 5519, 647: 5520, 811: 5554}, // 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, 790: 5525, 845: 5554}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 5525, 3148, 461: 3128, 3146, 2658, 3788, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 5524, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3784, 784: 5526, 827: 5555}, {736, 736, 7: 736}, - {768, 768, 7: 5570}, + {768, 768, 7: 5571}, {757, 757, 7: 757}, - {338: 5562}, + {338: 5563}, // 3110 - {149: 5560, 674: 5559}, + {149: 5561, 674: 5560}, {754, 754, 7: 754}, - {753, 753, 7: 753, 660: 5286, 908: 5561}, + {753, 753, 7: 753, 660: 5287, 908: 5562}, {752, 752, 7: 752}, - {207: 5564, 368: 5566, 658: 5565, 1212: 5563}, + {208: 5565, 368: 5567, 658: 5566, 1212: 5564}, // 3115 {755, 755, 7: 755}, - {658: 5569}, - {308: 5567, 388: 5568}, + {658: 5570}, + {308: 5568, 388: 5569}, {748, 748, 7: 748}, {750, 750, 7: 750}, // 3120 {749, 749, 7: 749}, {751, 751, 7: 751}, - {173: 5557, 658: 5558, 905: 5571}, + {173: 5558, 658: 5559, 905: 5572}, {756, 756, 7: 756}, - {173: 5557, 482: 1672, 639: 1672, 647: 1672, 658: 5558, 905: 5556, 952: 5573}, + {173: 5558, 482: 1672, 639: 1672, 647: 1672, 658: 5559, 905: 5557, 953: 5574}, // 3125 - {769, 769, 7: 5570}, + {769, 769, 7: 5571}, {765, 765}, - {762, 762, 476: 5578}, + {762, 762, 476: 5579}, {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, 785: 5539, 806: 5540, 5538, 844: 5579}, - {760, 760, 7: 5541}, - {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 5533, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 5538, 648: 3378, 2665, 2666, 2664, 725: 5067, 787: 5540, 807: 5541, 5539, 845: 5580}, + {760, 760, 7: 5542}, + {14: 5587, 461: 5586, 1079: 5591}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5583}, + {482: 5584}, // 3135 - {14: 5586, 461: 5585, 1078: 5584}, + {14: 5587, 461: 5586, 1079: 5585}, {771, 771}, {708, 708}, - {459: 5587}, - {461: 5145, 872: 5588}, + {459: 5588}, + {461: 5146, 877: 5589}, // 3140 - {57: 5589}, + {57: 5590}, {707, 707}, {772, 772}, - {731, 731, 7: 731, 467: 5593}, + {731, 731, 7: 731, 467: 5594}, {728, 728, 7: 728}, // 3145 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5594, 648: 3377, 2664, 2665, 2663, 725: 5595}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 464: 5595, 648: 3378, 2665, 2666, 2664, 725: 5596}, {730, 730, 7: 730}, {729, 729, 7: 729}, - {482: 5518, 647: 5519, 810: 5597}, - {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}, + {482: 5519, 647: 5520, 811: 5598}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 5525, 3148, 461: 3128, 3146, 2658, 3788, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 5524, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3784, 784: 5526, 827: 5599}, // 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, 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, 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}, + {734, 734, 7: 734}, + {173: 5558, 482: 1672, 639: 1672, 647: 1672, 658: 5559, 905: 5557, 953: 5601}, + {770, 770, 7: 5571}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5605, 2665, 2666, 2664, 887: 5612}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5605, 2665, 2666, 2664, 887: 5604}, // 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, 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: 5519, 647: 5520, 811: 5610}, + {471: 5607, 482: 740, 639: 5606, 647: 740}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5605, 2665, 2666, 2664, 887: 5609}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5605, 2665, 2666, 2664, 887: 5608}, {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, 790: 5525, 845: 5610}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 5525, 3148, 461: 3128, 3146, 2658, 3788, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 5524, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3784, 784: 5526, 827: 5611}, {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, 790: 5525, 845: 5613}, + {482: 5519, 647: 5520, 811: 5613}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 5525, 3148, 461: 3128, 3146, 2658, 3788, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 5524, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3784, 784: 5526, 827: 5614}, // 3165 {767, 767}, - {637: 5623}, - {637: 5616}, - {256: 5617}, - {482: 5618}, + {637: 5624}, + {637: 5617}, + {257: 5618}, + {482: 5619}, // 3170 - {461: 5619}, - {479: 5620}, - {255: 5621}, - {461: 5622}, + {461: 5620}, + {479: 5621}, + {256: 5622}, + {461: 5623}, {774, 774}, // 3175 - {256: 5624}, - {482: 5625}, - {461: 5626}, - {479: 5627}, - {255: 5628}, + {257: 5625}, + {482: 5626}, + {461: 5627}, + {479: 5628}, + {256: 5629}, // 3180 - {461: 5629}, + {461: 5630}, {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, 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}, + {459: 2498, 484: 2496, 553: 2495, 632: 2491, 696: 5642, 738: 5641, 2492, 2493, 2494, 5643}, + {459: 1201, 484: 1201, 553: 1201, 632: 1201, 638: 3433, 732: 3431, 3432, 768: 5635, 774: 5636, 917: 5638, 947: 5640}, + {459: 1201, 484: 1201, 553: 1201, 632: 1201, 638: 3433, 732: 3431, 3432, 768: 5635, 774: 5636, 917: 5638, 947: 5639}, // 3185 - {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: 1201, 484: 1201, 553: 1201, 632: 1201, 638: 3433, 732: 3431, 3432, 768: 5635, 774: 5636, 917: 5638, 947: 5637}, + {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, 491: 1204, 493: 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, 746: 1204, 755: 1204, 1204, 1204}, {459: 1200, 484: 1200, 553: 1200, 632: 1200}, {459: 777, 484: 777, 553: 777, 632: 777}, {459: 776, 484: 776, 553: 776, 632: 776}, @@ -9834,244 +9835,244 @@ var ( {459: 778, 484: 778, 553: 778, 632: 778}, {459: 779, 484: 779, 553: 779, 632: 779}, {791, 791, 57: 791, 458: 791, 460: 791, 466: 781, 468: 791, 476: 781, 781}, - {790, 790, 57: 790, 458: 790, 460: 790, 466: 780, 468: 790, 476: 780, 780, 480: 2630, 488: 2631, 490: 2627, 753: 5643, 5644}, + {790, 790, 57: 790, 458: 790, 460: 790, 466: 780, 468: 790, 476: 780, 780, 480: 2631, 488: 2632, 490: 2628, 753: 5644, 5645}, {466: 782, 476: 782, 782}, // 3195 - {789, 789, 57: 789, 458: 789, 460: 789, 468: 789, 480: 2630, 488: 2631, 754: 5645}, + {789, 789, 57: 789, 458: 789, 460: 789, 468: 789, 480: 2631, 488: 2632, 754: 5646}, {788, 788, 57: 788, 458: 788, 460: 788, 468: 788}, {787, 787, 57: 787, 458: 787, 460: 787, 468: 787}, - {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}, + {466: 780, 476: 780, 780, 480: 2631, 488: 2632, 490: 2628, 753: 3776, 3777}, + {7: 5662, 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, 914: 5649, 1141: 5660}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5651, 2665, 2666, 2664, 914: 5650, 1141: 5661}, {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, 856: 5655}, - {465: 5653}, + {459: 5652, 465: 2249, 1200: 5653}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5657, 2665, 2666, 2664, 858: 5656}, + {465: 5654}, // 3205 - {459: 2497, 696: 5654}, + {459: 2498, 696: 5655}, {7: 958, 459: 958, 484: 958, 553: 958, 632: 958, 641: 958, 731: 958}, - {7: 5658, 57: 5657}, - {2246, 2246, 7: 2246, 57: 2246, 460: 2246}, - {465: 2247}, + {7: 5659, 57: 5658}, + {2247, 2247, 7: 2247, 57: 2247, 460: 2247}, + {465: 2248}, // 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}, - {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, 914: 5662}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5660, 2665, 2666, 2664}, + {2246, 2246, 7: 2246, 57: 2246, 460: 2246}, + {7: 5662, 459: 961, 484: 961, 553: 961, 632: 961, 641: 961, 731: 961}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5651, 2665, 2666, 2664, 914: 5663}, {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, 867: 5666}, - {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3822, 868: 5667}, + {1258, 1258, 57: 1258, 458: 1258, 460: 1258, 466: 1258, 468: 1258, 476: 1258, 1258, 479: 1258, 1258, 1258, 483: 1258, 488: 1258, 490: 2628, 753: 2629, 797: 5665}, + {834, 834, 57: 834, 458: 834, 460: 834, 466: 834, 468: 834, 476: 834, 834, 479: 834, 2631, 834, 483: 834, 488: 2632, 754: 2633, 815: 5666}, + {806, 806, 57: 806, 458: 806, 460: 806, 466: 806, 468: 806, 476: 806, 806, 479: 3797, 481: 806, 483: 3798, 871: 5667}, + {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3823, 872: 5668}, {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, 867: 5670}, - {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3822, 868: 5671}, + {834, 834, 57: 834, 458: 834, 460: 834, 466: 834, 468: 834, 476: 834, 834, 479: 834, 2631, 834, 483: 834, 488: 2632, 754: 2633, 815: 5670}, + {806, 806, 57: 806, 458: 806, 460: 806, 466: 806, 468: 806, 476: 806, 806, 479: 3797, 481: 806, 483: 3798, 871: 5671}, + {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3823, 872: 5672}, {967, 967, 57: 967, 458: 967, 460: 967, 466: 967, 468: 967, 476: 967, 967}, - {646: 5679}, + {646: 5680}, // 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}, + {1258, 1258, 57: 1258, 458: 1258, 460: 1258, 466: 1258, 468: 1258, 476: 1258, 1258, 479: 1258, 1258, 1258, 483: 1258, 488: 1258, 490: 2628, 753: 2629, 797: 5676}, {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, 867: 5677}, - {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3822, 868: 5678}, + {834, 834, 57: 834, 458: 834, 460: 834, 466: 834, 468: 834, 476: 834, 834, 479: 834, 2631, 834, 483: 834, 488: 2632, 754: 2633, 815: 5677}, + {806, 806, 57: 806, 458: 806, 460: 806, 466: 806, 468: 806, 476: 806, 806, 479: 3797, 481: 806, 483: 3798, 871: 5678}, + {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3823, 872: 5679}, // 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, 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, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 2657, 852: 3130, 885: 5681}, + {1813, 1813, 7: 3299, 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, 2625, 237, 499: 237, 777: 2626, 5707}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 5231, 565: 5226, 648: 3806, 2665, 2666, 2664, 696: 5230, 724: 5229, 785: 5228, 788: 5227, 5233, 835: 5223, 874: 5692, 1179: 5691, 1292: 5690}, // 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, 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, 867: 5687}, - {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3822, 868: 5688}, + {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: 5673, 923: 5675, 946: 5685}, + {1258, 1258, 57: 1258, 458: 1258, 460: 1258, 466: 1258, 468: 1258, 476: 1258, 1258, 479: 1258, 1258, 1258, 483: 1258, 488: 1258, 490: 2628, 753: 2629, 797: 5686}, + {834, 834, 57: 834, 458: 834, 460: 834, 466: 834, 468: 834, 476: 834, 834, 479: 834, 2631, 834, 483: 834, 488: 2632, 754: 2633, 815: 5687}, + {806, 806, 57: 806, 458: 806, 460: 806, 466: 806, 468: 806, 476: 806, 806, 479: 3797, 481: 806, 483: 3798, 871: 5688}, + {811, 811, 57: 811, 458: 811, 460: 811, 466: 811, 468: 811, 476: 811, 811, 481: 3823, 872: 5689}, // 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, 777: 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, 2625, 237, 499: 237, 501: 237, 237, 777: 2626, 5693}, {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, 923: 5674, 946: 5693}, + {897, 897, 7: 5277, 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: 5673, 501: 813, 813, 923: 5675, 946: 5694}, // 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, 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}, + {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, 5695, 1199: 5696}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 5706}, + {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: 5698, 1315: 5697}, {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, 907: 5700, 1137: 5699, 1316: 5698}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3287, 2665, 2666, 2664, 907: 5701, 1137: 5700, 1316: 5699}, // 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}, + {955, 955, 7: 5704, 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, 1139: 5702}, + {465: 5702}, + {459: 3288, 1139: 5703}, {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, 907: 5700, 1137: 5704}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3287, 2665, 2666, 2664, 907: 5701, 1137: 5705}, {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}, + {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, 492: 3235, 494: 3233, 3234, 3232, 3230, 501: 1811, 720: 3231, 3229}, {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, 1026: 5725, 1188: 5726, 1261: 5727}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 509: 5724, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 5725, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 5723, 1027: 5726, 1188: 5727, 1261: 5728}, // 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}, - {2: 830, 830, 830, 830, 830, 8: 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 58: 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 459: 830, 461: 830, 830, 830, 830, 469: 830, 830, 830, 830, 830, 478: 830, 484: 830, 486: 830, 492: 830, 830, 500: 830, 509: 830, 529: 830, 552: 830, 554: 830, 830, 830, 830, 830, 830, 830, 830, 830, 564: 830, 830, 830, 830, 830, 830, 572: 830, 574: 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 636: 830, 638: 830, 732: 830, 830, 735: 830, 830, 830, 743: 830, 755: 830, 830, 830}, - {2: 829, 829, 829, 829, 829, 8: 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 58: 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 459: 829, 461: 829, 829, 829, 829, 469: 829, 829, 829, 829, 829, 478: 829, 484: 829, 486: 829, 492: 829, 829, 500: 829, 509: 829, 529: 829, 552: 829, 554: 829, 829, 829, 829, 829, 829, 829, 829, 829, 564: 829, 829, 829, 829, 829, 829, 572: 829, 574: 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 636: 829, 638: 829, 732: 829, 829, 735: 829, 829, 829, 743: 829, 755: 829, 829, 829}, - {2: 828, 828, 828, 828, 828, 8: 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 58: 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 459: 828, 461: 828, 828, 828, 828, 469: 828, 828, 828, 828, 828, 478: 828, 484: 828, 486: 828, 492: 828, 828, 500: 828, 509: 828, 529: 828, 552: 828, 554: 828, 828, 828, 828, 828, 828, 828, 828, 828, 564: 828, 828, 828, 828, 828, 828, 572: 828, 574: 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 636: 828, 638: 828, 732: 828, 828, 735: 828, 828, 828, 743: 828, 755: 828, 828, 828}, + {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, 491: 832, 493: 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, 746: 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, 491: 831, 493: 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, 746: 831, 755: 831, 831, 831}, + {2: 830, 830, 830, 830, 830, 8: 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 58: 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 459: 830, 461: 830, 830, 830, 830, 469: 830, 830, 830, 830, 830, 478: 830, 484: 830, 486: 830, 491: 830, 493: 830, 500: 830, 509: 830, 529: 830, 552: 830, 554: 830, 830, 830, 830, 830, 830, 830, 830, 830, 564: 830, 830, 830, 830, 830, 830, 572: 830, 574: 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 636: 830, 638: 830, 732: 830, 830, 735: 830, 830, 830, 746: 830, 755: 830, 830, 830}, + {2: 829, 829, 829, 829, 829, 8: 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 58: 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 459: 829, 461: 829, 829, 829, 829, 469: 829, 829, 829, 829, 829, 478: 829, 484: 829, 486: 829, 491: 829, 493: 829, 500: 829, 509: 829, 529: 829, 552: 829, 554: 829, 829, 829, 829, 829, 829, 829, 829, 829, 564: 829, 829, 829, 829, 829, 829, 572: 829, 574: 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 636: 829, 638: 829, 732: 829, 829, 735: 829, 829, 829, 746: 829, 755: 829, 829, 829}, + {2: 828, 828, 828, 828, 828, 8: 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 58: 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 459: 828, 461: 828, 828, 828, 828, 469: 828, 828, 828, 828, 828, 478: 828, 484: 828, 486: 828, 491: 828, 493: 828, 500: 828, 509: 828, 529: 828, 552: 828, 554: 828, 828, 828, 828, 828, 828, 828, 828, 828, 564: 828, 828, 828, 828, 828, 828, 572: 828, 574: 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 636: 828, 638: 828, 732: 828, 828, 735: 828, 828, 828, 746: 828, 755: 828, 828, 828}, // 3265 - {2: 827, 827, 827, 827, 827, 8: 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 58: 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 459: 827, 461: 827, 827, 827, 827, 469: 827, 827, 827, 827, 827, 478: 827, 484: 827, 486: 827, 492: 827, 827, 500: 827, 509: 827, 529: 827, 552: 827, 554: 827, 827, 827, 827, 827, 827, 827, 827, 827, 564: 827, 827, 827, 827, 827, 827, 572: 827, 574: 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 636: 827, 638: 827, 732: 827, 827, 735: 827, 827, 827, 743: 827, 755: 827, 827, 827}, - {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, 771: 5709, 824: 5710, 847: 5708, 1100: 5721, 5714}, + {2: 827, 827, 827, 827, 827, 8: 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 58: 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 459: 827, 461: 827, 827, 827, 827, 469: 827, 827, 827, 827, 827, 478: 827, 484: 827, 486: 827, 491: 827, 493: 827, 500: 827, 509: 827, 529: 827, 552: 827, 554: 827, 827, 827, 827, 827, 827, 827, 827, 827, 564: 827, 827, 827, 827, 827, 827, 572: 827, 574: 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 636: 827, 638: 827, 732: 827, 827, 735: 827, 827, 827, 746: 827, 755: 827, 827, 827}, + {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, 491: 826, 493: 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, 746: 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, 491: 825, 493: 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, 746: 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, 491: 824, 493: 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, 746: 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, 5714, 5720, 5721, 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, 491: 822, 493: 822, 500: 5717, 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: 3433, 732: 3431, 3432, 735: 5219, 5218, 5217, 746: 5214, 755: 5713, 5716, 5712, 768: 5635, 774: 5710, 824: 5711, 847: 5709, 1101: 5722, 5715}, // 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, 1185: 5736, 5735}, + {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, 491: 820, 493: 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, 746: 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, 491: 816, 493: 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, 746: 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, 491: 815, 493: 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, 746: 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, 491: 821, 493: 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, 746: 821, 755: 821, 821, 821}, + {1821, 1821, 2901, 2749, 2785, 2903, 2676, 1821, 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 1821, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 1821, 460: 1821, 5740, 465: 5739, 1821, 468: 1821, 476: 1821, 1821, 479: 1821, 1821, 1821, 483: 1821, 487: 1821, 1821, 1821, 1821, 492: 3235, 494: 3233, 3234, 3232, 3230, 499: 1821, 648: 5738, 2665, 2666, 2664, 720: 3231, 3229, 1185: 5737, 5736}, // 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}, + {1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 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, 492: 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: 5731, 642: 1241, 1241}, {1815, 1815, 7: 1815, 57: 1815, 458: 1815, 460: 1815, 466: 1815, 468: 1815, 476: 1815, 1815, 479: 1815, 1815, 1815, 483: 1815, 487: 1815, 1815, 1815, 1815, 499: 1815}, - {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}, + {814, 814, 7: 5729, 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, 1026: 5729}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 509: 5724, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 5725, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 5723, 1027: 5730}, {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 509: 5732, 648: 5733, 2665, 2666, 2664}, {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}, - {1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 3873, 1240, 1240, 1240, 1240, 465: 1240, 1240, 1240, 1240, 471: 1240, 1240, 1240, 476: 1240, 1240, 479: 1240, 1240, 1240, 1240, 1240, 487: 1240, 1240, 1240, 1240, 1240, 494: 1240, 1240, 1240, 1240, 499: 1240, 507: 1240, 509: 1240, 530: 1240, 533: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 570: 1240, 639: 5733, 642: 1240, 1240}, + {1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 3874, 1240, 1240, 1240, 1240, 465: 1240, 1240, 1240, 1240, 471: 1240, 1240, 1240, 476: 1240, 1240, 479: 1240, 1240, 1240, 1240, 1240, 487: 1240, 1240, 1240, 1240, 492: 1240, 494: 1240, 1240, 1240, 1240, 499: 1240, 507: 1240, 509: 1240, 530: 1240, 533: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 570: 1240, 639: 5734, 642: 1240, 1240}, // 3285 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5734, 648: 3670, 2664, 2665, 2663}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 509: 5735, 648: 3671, 2665, 2666, 2664}, {1823, 1823, 7: 1823, 57: 1823, 458: 1823, 460: 1823, 466: 1823, 468: 1823, 476: 1823, 1823, 479: 1823, 1823, 1823, 483: 1823, 487: 1823, 1823, 1823, 1823, 499: 1823}, {1822, 1822, 7: 1822, 57: 1822, 458: 1822, 460: 1822, 466: 1822, 468: 1822, 476: 1822, 1822, 479: 1822, 1822, 1822, 483: 1822, 487: 1822, 1822, 1822, 1822, 499: 1822}, {1820, 1820, 7: 1820, 57: 1820, 458: 1820, 460: 1820, 466: 1820, 468: 1820, 476: 1820, 1820, 479: 1820, 1820, 1820, 483: 1820, 487: 1820, 1820, 1820, 1820, 499: 1820}, {1819, 1819, 7: 1819, 57: 1819, 458: 1819, 460: 1819, 466: 1819, 468: 1819, 476: 1819, 1819, 479: 1819, 1819, 1819, 483: 1819, 487: 1819, 1819, 1819, 1819, 499: 1819}, // 3290 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5741, 648: 5740, 2664, 2665, 2663}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 5742, 648: 5741, 2665, 2666, 2664}, {1817, 1817, 7: 1817, 57: 1817, 458: 1817, 460: 1817, 466: 1817, 468: 1817, 476: 1817, 1817, 479: 1817, 1817, 1817, 483: 1817, 487: 1817, 1817, 1817, 1817, 499: 1817}, {1818, 1818, 7: 1818, 57: 1818, 458: 1818, 460: 1818, 466: 1818, 468: 1818, 476: 1818, 1818, 479: 1818, 1818, 1818, 483: 1818, 487: 1818, 1818, 1818, 1818, 499: 1818}, {1816, 1816, 7: 1816, 57: 1816, 458: 1816, 460: 1816, 466: 1816, 468: 1816, 476: 1816, 1816, 479: 1816, 1816, 1816, 483: 1816, 487: 1816, 1816, 1816, 1816, 499: 1816}, {985, 985}, // 3295 {995, 995}, - {81: 5749, 221: 5748}, + {81: 5750, 222: 5749}, {989, 989}, - {843: 5747}, + {844: 5748}, {988, 988}, // 3300 - {991, 991, 81: 5754}, - {221: 5750}, - {990, 990, 81: 5752, 843: 5751}, + {991, 991, 81: 5755}, + {222: 5751}, + {990, 990, 81: 5753, 844: 5752}, {993, 993}, - {843: 5753}, + {844: 5754}, // 3305 {992, 992}, - {843: 5755}, + {844: 5756}, {994, 994}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5757, 2664, 2665, 2663}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5758, 2665, 2666, 2664}, {999, 999}, // 3310 - {1003, 1003, 468: 5759}, - {554: 3237, 697: 5761, 1302: 5760}, - {1002, 1002, 7: 5762}, + {1003, 1003, 468: 5760}, + {554: 3238, 697: 5762, 1302: 5761}, + {1002, 1002, 7: 5763}, {1001, 1001, 7: 1001}, - {554: 3237, 697: 5763}, + {554: 3238, 697: 5764}, // 3315 {1000, 1000, 7: 1000}, - {487: 5765}, - {461: 5767, 554: 3237, 697: 5768, 1253: 5766}, + {487: 5766}, + {461: 5768, 554: 3238, 697: 5769, 1253: 5767}, {1006, 1006}, {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, 1044: 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: 5771, 1045: 5772}, {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5773}, + {150: 887, 459: 887, 887, 475: 5281, 484: 887, 498: 887, 553: 887, 632: 887, 843: 5774}, // 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, 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}, + {150: 5782, 459: 5775, 2497, 484: 5783, 498: 5781, 553: 2495, 632: 2491, 696: 5780, 738: 5778, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 5779, 5777, 3765, 959: 5776, 1044: 5784}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 2251, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 2498, 2497, 484: 2496, 553: 2495, 632: 2491, 648: 4025, 2665, 2666, 2664, 696: 5647, 729: 4026, 738: 3766, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 3768, 3767, 3765, 795: 4954, 983: 5796}, + {459: 3782, 826: 5793, 957: 5792}, {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, 986: 5786, 1168: 5784}, + {1305, 1305, 458: 1305, 466: 780, 468: 1305, 476: 780, 780, 480: 2631, 488: 2632, 490: 2628, 753: 3776, 3777}, + {1291, 1291, 2901, 2749, 2785, 2903, 2676, 1291, 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 1291, 468: 1291, 648: 4025, 2665, 2666, 2664, 729: 5786, 987: 5787, 1168: 5785}, {459: 1303}, - {459: 1302, 561: 3780, 896: 3779, 957: 3778}, + {459: 1302, 561: 3781, 897: 3780, 958: 3779}, // 3335 {1286, 1286, 468: 1286}, - {1304, 1304, 7: 5789, 458: 1304, 468: 1304}, - {482: 5787}, + {1304, 1304, 7: 5790, 458: 1304, 468: 1304}, + {482: 5788}, {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, 790: 5788}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3788, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3784, 784: 5789}, // 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, 986: 5790}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 5786, 987: 5791}, {1289, 1289, 7: 1289, 458: 1289, 468: 1289}, - {1309, 1309, 7: 5793, 458: 1309, 468: 1309}, + {1309, 1309, 7: 5794, 458: 1309, 468: 1309}, {1301, 1301, 7: 1301, 458: 1301, 468: 1301}, // 3345 - {459: 3781, 826: 5794}, + {459: 3782, 826: 5795}, {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, 958: 5797}, - {459: 3781, 826: 5792, 956: 5802}, + {57: 5797}, + {150: 5782, 459: 2498, 2497, 484: 5783, 553: 2495, 632: 2491, 696: 5802, 738: 5800, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 5801, 5799, 3765, 959: 5798}, + {459: 3782, 826: 5793, 957: 5803}, // 3350 {1313, 1313, 458: 1313, 468: 1313}, {1312, 1312, 458: 1312, 466: 781, 468: 1312, 476: 781, 781}, {1311, 1311, 458: 1311, 468: 1311}, - {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}, + {1310, 1310, 458: 1310, 466: 780, 468: 1310, 476: 780, 780, 480: 2631, 488: 2632, 490: 2628, 753: 3776, 3777}, + {1314, 1314, 7: 5794, 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, 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, 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}, + {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: 5219, 5218, 5217, 824: 5220, 869: 5805}, + {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: 4072, 481: 1803, 839: 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: 5771, 1045: 5807}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5808}, + {150: 887, 459: 887, 887, 475: 5281, 484: 887, 498: 887, 553: 887, 632: 887, 843: 5809}, // 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, 958: 5775, 1043: 5809}, - {1288, 1288, 458: 5811, 468: 1288, 1232: 5810}, + {150: 5782, 459: 5775, 2497, 484: 5783, 498: 5781, 553: 2495, 632: 2491, 696: 5780, 738: 5778, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 5779, 5777, 3765, 959: 5776, 1044: 5810}, + {1288, 1288, 458: 5812, 468: 1288, 1232: 5811}, {1317, 1317, 468: 1317}, - {192: 5812}, - {563: 5813}, + {192: 5813}, + {563: 5814}, // 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, 871: 5335, 909: 5815}, - {1287, 1287, 7: 5337, 468: 1287}, - {1321, 1321, 459: 5824, 639: 1779}, + {641: 5815}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 5335, 876: 5336, 909: 5816}, + {1287, 1287, 7: 5338, 468: 1287}, + {1321, 1321, 459: 5825, 639: 1779}, {1322, 1322}, // 3370 - {639: 5819}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5820, 2664, 2665, 2663}, - {1320, 1320, 459: 5821}, - {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, 1866, 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: 3636, 811: 5822}, - {57: 5823}, + {639: 5820}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5821, 2665, 2666, 2664}, + {1320, 1320, 459: 5822}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 1866, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 761: 3637, 812: 5823}, + {57: 5824}, // 3375 {1318, 1318}, - {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, 1866, 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: 3636, 811: 5825}, - {57: 5826}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 1866, 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 3475, 761: 3637, 812: 5826}, + {57: 5827}, {1319, 1319}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5977, 2664, 2665, 2663}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5978, 2665, 2666, 2664}, // 3380 - {577, 577, 489: 5974, 507: 5973, 1268: 5972}, - {26: 5960, 99: 5957, 132: 5962, 158: 5961, 185: 5959, 553: 5956, 568: 5958}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 5945, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5946}, - {648, 648, 479: 5940}, - {125: 5939}, + {577, 577, 489: 5975, 507: 5974, 1268: 5973}, + {16: 5961, 99: 5958, 132: 5963, 160: 5962, 185: 5960, 553: 5957, 568: 5959}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 5946, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5947}, + {648, 648, 479: 5941}, + {125: 5940}, // 3385 - {98: 3828, 107: 3827, 110: 5934, 204: 5933, 820: 5935}, + {98: 3829, 107: 3828, 110: 5935, 205: 5934, 820: 5936}, {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, 1106: 5913, 1271: 5912, 5911}, + {636, 636, 176: 5916, 218: 5917, 228: 5918, 231: 5915, 249: 5920, 259: 5919, 273: 5922, 278: 5921, 479: 636, 636, 488: 636, 638: 5923, 1107: 5914, 1271: 5913, 5912}, {642, 642}, {641, 641}, // 3390 - {579, 579, 251: 5902, 479: 5901, 489: 579, 507: 579}, + {579, 579, 252: 5903, 479: 5902, 489: 579, 507: 579}, {487: 619, 530: 619}, {487: 618, 530: 618}, {487: 617, 530: 617}, @@ -10080,22 +10081,22 @@ var ( {613, 613, 489: 613, 507: 613}, {612, 612, 489: 612, 507: 612}, {611, 611, 489: 611, 507: 611}, - {110: 5899}, - {487: 5875, 530: 5876, 791: 5894}, + {110: 5900}, + {487: 5876, 530: 5877, 792: 5895}, // 3400 - {98: 571, 107: 571, 197: 5873, 1068: 5888}, + {98: 571, 107: 571, 197: 5874, 1069: 5889}, {603, 603, 489: 603, 507: 603}, {602, 602, 489: 602, 507: 602}, - {125: 5886, 138: 5887, 189: 5885}, + {125: 5887, 138: 5888, 189: 5886}, {598, 598, 489: 598, 507: 598}, // 3405 - {569, 569, 487: 5875, 489: 569, 507: 569, 530: 5876, 791: 5878, 827: 5884}, + {569, 569, 487: 5876, 489: 569, 507: 569, 530: 5877, 792: 5879, 828: 5885}, + {125: 5884}, {125: 5883}, {125: 5882}, {125: 5881}, - {125: 5880}, // 3410 - {569, 569, 487: 5875, 489: 569, 507: 569, 530: 5876, 791: 5878, 827: 5877}, + {569, 569, 487: 5876, 489: 569, 507: 569, 530: 5877, 792: 5879, 828: 5878}, {591, 591, 489: 591, 507: 591}, {590, 590, 489: 590, 507: 590}, {589, 589, 489: 589, 507: 589}, @@ -10105,7 +10106,7 @@ var ( {586, 586, 489: 586, 507: 586}, {585, 585, 489: 585, 507: 585}, {584, 584, 489: 584, 507: 584}, - {125: 5874}, + {125: 5875}, // 3420 {582, 582, 489: 582, 507: 582}, {581, 581, 489: 581, 507: 581}, @@ -10113,13 +10114,13 @@ var ( {125: 573, 138: 573, 189: 573}, {125: 572, 138: 572, 151: 572, 189: 572}, // 3425 - {98: 570, 107: 570, 110: 570, 204: 570}, + {98: 570, 107: 570, 110: 570, 205: 570}, {583, 583, 489: 583, 507: 583}, {2: 616, 616, 616, 616, 616, 8: 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 58: 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616, 616}, {2: 615, 615, 615, 615, 615, 8: 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 58: 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615, 615}, {592, 592, 489: 592, 507: 592}, // 3430 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5879}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5212, 2665, 2666, 2664, 801: 5880}, {568, 568, 489: 568, 507: 568}, {593, 593, 489: 593, 507: 593}, {594, 594, 489: 594, 507: 594}, @@ -10131,43 +10132,43 @@ var ( {600, 600, 489: 600, 507: 600}, {599, 599, 489: 599, 507: 599}, // 3440 - {98: 3828, 107: 3827, 820: 5889}, - {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}, + {98: 3829, 107: 3828, 820: 5890}, + {487: 5876, 530: 5877, 792: 5892, 1109: 5891}, + {569, 569, 487: 5876, 489: 569, 507: 569, 530: 5877, 792: 5879, 828: 5894}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5893}, {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, 791: 5897}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5896, 2665, 2666, 2664, 724: 5897}, + {1018, 1018, 487: 5876, 489: 1018, 507: 1018, 530: 5877, 639: 3811, 792: 5898}, {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5899, 2665, 2666, 2664}, // 3450 {606, 606, 489: 606, 507: 606}, - {569, 569, 487: 5875, 489: 569, 507: 569, 530: 5876, 791: 5878, 827: 5900}, + {569, 569, 487: 5876, 489: 569, 507: 569, 530: 5877, 792: 5879, 828: 5901}, {609, 609, 489: 609, 507: 609}, - {553: 5906, 568: 5903, 833: 5905, 1269: 5904}, + {553: 5907, 568: 5904, 834: 5906, 1269: 5905}, {578, 578, 489: 578, 507: 578}, // 3455 - {2: 2008, 2008, 2008, 2008, 2008, 8: 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 58: 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 2008, 464: 2008, 467: 2008, 486: 2008, 509: 2008, 552: 2008, 634: 2008}, + {2: 2009, 2009, 2009, 2009, 2009, 8: 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 58: 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 2009, 464: 2009, 467: 2009, 486: 2009, 509: 2009, 552: 2009, 634: 2009}, {640, 640}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5910}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5907}, - {638, 638, 475: 5908}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5212, 2665, 2666, 2664, 801: 5911}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5908}, + {638, 638, 475: 5909}, // 3460 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5909, 2664, 2665, 2663}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5910, 2665, 2666, 2664}, {637, 637}, {639, 639}, - {623, 623, 479: 5929, 623, 488: 623, 1270: 5928}, - {635, 635, 7: 5926, 479: 635, 635, 488: 635}, + {623, 623, 479: 5930, 623, 488: 623, 1270: 5929}, + {635, 635, 7: 5927, 479: 635, 635, 488: 635}, // 3465 {634, 634, 7: 634, 479: 634, 634, 488: 634}, {632, 632, 7: 632, 479: 632, 632, 488: 632}, {631, 631, 7: 631, 479: 631, 631, 488: 631}, - {334: 5925}, - {375: 5924}, + {334: 5926}, + {375: 5925}, // 3470 - {323: 5923}, + {323: 5924}, {627, 627, 7: 627, 479: 627, 627, 488: 627}, {626, 626, 7: 626, 479: 626, 626, 488: 626}, {625, 625, 7: 625, 479: 625, 625, 488: 625}, @@ -10176,104 +10177,104 @@ 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, 1106: 5927}, + {176: 5916, 218: 5917, 228: 5918, 231: 5915, 249: 5920, 259: 5919, 273: 5922, 278: 5921, 638: 5923, 1107: 5928}, {633, 633, 7: 633, 479: 633, 633, 488: 633}, // 3480 - {834, 834, 480: 2630, 488: 2631, 754: 2632, 815: 5932}, - {143: 5930}, - {493: 2638, 722: 3934, 748: 5931}, + {834, 834, 480: 2631, 488: 2632, 754: 2633, 815: 5933}, + {143: 5931}, + {493: 2639, 722: 3935, 748: 5932}, {622, 622, 480: 622, 488: 622}, {643, 643}, // 3485 {645, 645}, - {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}, + {569, 569, 487: 5876, 489: 569, 507: 569, 530: 5877, 792: 5879, 828: 5939}, + {487: 5876, 530: 5877, 792: 5892, 1109: 5937}, + {569, 569, 487: 5876, 489: 569, 507: 569, 530: 5877, 792: 5879, 828: 5938}, {605, 605, 489: 605, 507: 605}, // 3490 {610, 610, 489: 610, 507: 610}, {646, 646}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5941}, - {621, 621, 468: 5943, 1303: 5942}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5942}, + {621, 621, 468: 5944, 1303: 5943}, {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, 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, 791: 5878, 827: 5955}, - {15: 887, 153: 5948, 475: 5280, 635: 887, 842: 5947}, - {15: 5949, 635: 5950}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 5533, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 5538, 648: 3378, 2665, 2666, 2664, 725: 5067, 787: 5540, 807: 5541, 5539, 845: 5945}, + {620, 620, 7: 5542}, + {569, 569, 15: 1683, 154: 1683, 475: 1683, 487: 5876, 489: 569, 507: 569, 530: 5877, 635: 1683, 639: 1683, 792: 5879, 828: 5956}, + {15: 887, 154: 5949, 475: 5281, 635: 887, 843: 5948}, + {15: 5950, 635: 5951}, // 3500 {650, 650}, - {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, 777: 2625, 5953}, + {237, 237, 489: 2625, 777: 2626, 5955}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5952, 2665, 2666, 2664}, + {15: 5953}, + {237, 237, 489: 2625, 777: 2626, 5954}, // 3505 {649, 649}, {651, 651}, {608, 608, 489: 608, 507: 608}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5971}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5970}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5972}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5971}, // 3510 - {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: 5968}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5967}, - {154: 5965}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5964}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5963, 2664, 2665, 2663}, + {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: 4702, 769: 5969}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 5968}, + {155: 5966}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5965}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5964, 2665, 2666, 2664}, // 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, 865: 5966}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4862, 2665, 2666, 2664, 868: 5967}, {654, 654}, {655, 655}, // 3520 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5969}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5212, 2665, 2666, 2664, 801: 5970}, {656, 656}, {657, 657}, {658, 658}, {659, 659}, // 3525 - {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, 5976, 3140, 3223, 3139, 3136}, - {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, 5975}, - {575, 575, 491: 3234, 494: 3232, 3233, 3231, 3229, 720: 3230, 3228}, - {576, 576, 467: 3238, 570: 3239}, - {1902, 1902, 198: 5979, 553: 1902, 1235: 5978}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 464: 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 3238, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 5977, 3141, 3224, 3140, 3137}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 2690, 2742, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 2771, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 2669, 2685, 2828, 2919, 2776, 2703, 2720, 2847, 2930, 2763, 2732, 2841, 2842, 2837, 2797, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 2778, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 2782, 2680, 2729, 2663, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 2701, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 2767, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 2768, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 2836, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 2654, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 2784, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 2726, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 2655, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 2679, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3052, 3053, 3101, 3100, 2956, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 2818, 2835, 2957, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3080, 3081, 3091, 3077, 3078, 3079, 3109, 2779, 459: 3148, 461: 3128, 3146, 2658, 3156, 469: 3161, 3165, 3144, 3145, 3183, 478: 3119, 484: 3157, 486: 3181, 491: 3164, 493: 3123, 529: 3152, 552: 3159, 554: 2656, 3182, 3166, 3118, 3120, 3122, 3121, 3149, 3126, 564: 3139, 3151, 3127, 3160, 3158, 3150, 572: 3155, 574: 3226, 3162, 3171, 3172, 3173, 3125, 3142, 3143, 3196, 3199, 3200, 3201, 3202, 3203, 3153, 3204, 3179, 3184, 3194, 3195, 3188, 3205, 3206, 3207, 3189, 3209, 3210, 3197, 3190, 3208, 3185, 3193, 3191, 3177, 3211, 3212, 3154, 3216, 3167, 3168, 3170, 3215, 3221, 3220, 3222, 3219, 3223, 3218, 3217, 3214, 3163, 3213, 3169, 3174, 3175, 636: 2659, 648: 3132, 2665, 2666, 2664, 696: 3147, 3225, 3133, 3138, 3124, 3198, 3136, 3134, 3135, 3176, 3187, 3186, 3180, 3178, 3192, 3131, 3141, 3224, 3140, 3137, 2662, 2661, 2660, 5976}, + {575, 575, 492: 3235, 494: 3233, 3234, 3232, 3230, 720: 3231, 3229}, + {576, 576, 467: 3239, 570: 3240}, + {1902, 1902, 198: 5980, 553: 1902, 1235: 5979}, // 3530 - {545, 545, 553: 5981, 950: 5980}, + {545, 545, 553: 5982, 950: 5981}, {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, 782: 5982}, - {544, 544, 7: 3808}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 3807, 782: 5983}, + {544, 544, 7: 3809}, // 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}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5988, 2664, 2665, 2663}, - {462: 3970, 3969, 796: 5986}, - {183: 5987}, + {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: 5986, 1201: 5985}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5989, 2665, 2666, 2664}, + {462: 3971, 3970, 796: 5987}, + {183: 5988}, {2: 1903, 1903, 1903, 1903, 1903, 8: 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 58: 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903, 1903}, // 3540 {1910, 1910}, - {2: 1906, 1906, 1906, 1906, 1906, 8: 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 58: 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 552: 5991, 1202: 5990}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 5993, 2664, 2665, 2663}, - {183: 5992}, + {2: 1906, 1906, 1906, 1906, 1906, 8: 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 58: 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 552: 5992, 1202: 5991}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5994, 2665, 2666, 2664}, + {183: 5993}, {2: 1905, 1905, 1905, 1905, 1905, 8: 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 58: 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905}, // 3545 {1911, 1911}, - {493: 2638, 722: 5995}, + {493: 2639, 722: 5996}, {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, 1177: 6001}, + {487: 6006}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 509: 6001, 648: 5212, 2665, 2666, 2664, 801: 6003, 1177: 6002}, // 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, 782: 5999}, - {7: 3808, 487: 1960, 637: 1960}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 3807, 782: 6000}, + {7: 3809, 487: 1960, 637: 1960}, {487: 1962, 637: 1962}, - {7: 6003, 487: 1961, 637: 1961}, + {7: 6004, 487: 1961, 637: 1961}, {7: 1959, 487: 1959, 637: 1959}, // 3555 - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5212, 2665, 2666, 2664, 801: 6005}, {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, 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}, + {461: 6007}, + {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, 883: 6008}, + {1963, 1963, 27: 6035, 59: 6011, 61: 6031, 6024, 6014, 6010, 6018, 6022, 6034, 6017, 6023, 6021, 6019, 6032, 6025, 6013, 6033, 6012, 6015, 6016, 6020, 458: 6026, 640: 6036, 879: 6028, 6027, 6030, 6009, 884: 6029}, // 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}, @@ -10293,22 +10294,22 @@ var ( {461: 1943, 482: 1943}, {461: 1942, 482: 1942}, // 3575 - {2: 1941, 1941, 1941, 1941, 1941, 8: 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 58: 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 461: 1941, 474: 1941, 482: 1941, 492: 1941}, - {2: 1940, 1940, 1940, 1940, 1940, 8: 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 58: 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 461: 1940, 474: 1940, 482: 1940, 492: 1940}, - {192: 6075}, - {482: 4171, 493: 1987, 723: 6073}, - {482: 4171, 493: 1987, 557: 1987, 1987, 723: 6071}, + {2: 1941, 1941, 1941, 1941, 1941, 8: 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 58: 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 461: 1941, 474: 1941, 482: 1941, 491: 1941}, + {2: 1940, 1940, 1940, 1940, 1940, 8: 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 58: 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 461: 1940, 474: 1940, 482: 1940, 491: 1940}, + {192: 6076}, + {482: 4172, 493: 1988, 723: 6074}, + {482: 4172, 493: 1988, 557: 1988, 1988, 723: 6072}, // 3580 - {461: 1987, 482: 4171, 723: 6069}, - {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, 474: 1987, 482: 4171, 492: 1987, 723: 6064}, - {461: 1987, 482: 4171, 493: 1987, 723: 6059}, - {461: 1987, 482: 4171, 493: 1987, 723: 6056}, - {482: 4171, 493: 1987, 723: 6051}, + {461: 1988, 482: 4172, 723: 6070}, + {2: 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, 461: 1988, 474: 1988, 482: 4172, 491: 1988, 723: 6065}, + {461: 1988, 482: 4172, 493: 1988, 723: 6060}, + {461: 1988, 482: 4172, 493: 1988, 723: 6057}, + {482: 4172, 493: 1988, 723: 6052}, // 3585 - {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, 911: 6037, 1071: 6038}, + {98: 1988, 107: 1988, 482: 4172, 493: 1988, 723: 6049}, + {177: 1988, 1988, 181: 1988, 482: 4172, 493: 1988, 557: 1988, 1988, 723: 6046}, + {177: 1988, 1988, 181: 1988, 482: 4172, 493: 1988, 557: 1988, 1988, 723: 6037}, + {177: 6043, 6044, 181: 6045, 493: 2639, 557: 6041, 6042, 722: 6040, 911: 6038, 1072: 6039}, {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,674 +10320,677 @@ 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, 911: 6046, 1071: 6047}, + {177: 6043, 6044, 181: 6045, 493: 2639, 557: 6041, 6042, 722: 6040, 911: 6047, 1072: 6048}, {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 - {98: 3828, 107: 3827, 493: 2638, 722: 2637, 730: 6050, 820: 6049}, + {98: 3829, 107: 3828, 493: 2639, 722: 2638, 730: 6051, 820: 6050}, {1928, 1928, 27: 1928, 59: 1928, 61: 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 458: 1928, 640: 1928}, {1927, 1927, 27: 1927, 59: 1927, 61: 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 458: 1927, 640: 1927}, - {493: 2638, 722: 2637, 730: 6052}, - {200: 6053}, + {493: 2639, 722: 2638, 730: 6053}, + {200: 6054}, // 3605 - {542: 6054}, - {108: 6055}, + {542: 6055}, + {108: 6056}, {1929, 1929, 27: 1929, 59: 1929, 61: 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 458: 1929, 640: 1929}, - {461: 6057, 493: 2638, 722: 2637, 730: 6058}, + {461: 6058, 493: 2639, 722: 2638, 730: 6059}, {1931, 1931, 27: 1931, 59: 1931, 61: 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 458: 1931, 640: 1931}, // 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, 792: 6062}, + {461: 6062, 493: 2639, 722: 2638, 730: 6061}, + {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: 3351, 3347, 108: 3344, 3359, 111: 3346, 3343, 3345, 3349, 3350, 3355, 3354, 3353, 3357, 3358, 3352, 3356, 3348, 458: 1932, 640: 1932, 793: 6063}, {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}, + {292: 6064}, // 3615 {1934, 1934, 27: 1934, 59: 1934, 61: 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 458: 1934, 640: 1934}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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, 474: 6067, 492: 6068, 648: 3377, 2664, 2665, 2663, 725: 6066, 1284: 6065}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 474: 6068, 491: 6069, 648: 3378, 2665, 2666, 2664, 725: 6067, 1284: 6066}, {1935, 1935, 27: 1935, 59: 1935, 61: 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 458: 1935, 640: 1935}, {246, 246, 27: 246, 59: 246, 61: 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 458: 246, 640: 246}, {245, 245, 27: 245, 59: 245, 61: 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 458: 245, 640: 245}, // 3620 {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}, + {461: 6071}, {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, 911: 6072}, + {493: 2639, 557: 6041, 6042, 722: 6040, 911: 6073}, {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}, + {493: 2639, 722: 2638, 730: 6075}, {1938, 1938, 27: 1938, 59: 1938, 61: 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 458: 1938, 640: 1938}, - {2: 1939, 1939, 1939, 1939, 1939, 8: 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 58: 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 461: 1939, 474: 1939, 482: 1939, 492: 1939}, - {637: 6077}, - {461: 6078}, + {2: 1939, 1939, 1939, 1939, 1939, 8: 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 58: 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 461: 1939, 474: 1939, 482: 1939, 491: 1939}, + {637: 6078}, + {461: 6079}, // 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, 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}, + {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, 883: 6080}, + {1964, 1964, 27: 6035, 59: 6011, 61: 6031, 6024, 6014, 6010, 6018, 6022, 6034, 6017, 6023, 6021, 6019, 6032, 6025, 6013, 6033, 6012, 6015, 6016, 6020, 458: 6026, 640: 6036, 879: 6028, 6027, 6030, 6009, 884: 6029}, + {1979, 1979, 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 6106}, {1977, 1977}, - {37: 6103}, + {37: 6104}, // 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, 773: 2615, 775: 2616, 2614, 789: 2466, 795: 2613, 808: 2619, 835: 6085}, + {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: 6087, 639: 1716}, + {459: 2498, 2497, 484: 2496, 491: 2482, 553: 2495, 555: 2481, 632: 2491, 641: 2595, 653: 2611, 696: 2612, 731: 2465, 738: 2613, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 2619, 2618, 2468, 763: 2594, 2466, 770: 2616, 772: 2617, 2615, 783: 2467, 791: 2614, 809: 2620, 836: 6086}, {1971, 1971}, - {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}, + {174: 6091, 298: 6094, 314: 6093, 385: 6090, 390: 6095, 461: 6088, 561: 6092, 1182: 6089}, + {459: 2498, 2497, 479: 6100, 484: 2496, 491: 2482, 553: 2495, 555: 2481, 632: 2491, 641: 2595, 653: 2611, 696: 2612, 731: 2465, 738: 2613, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 2619, 2618, 2468, 763: 2594, 2466, 770: 2616, 772: 2617, 2615, 783: 2467, 791: 2614, 809: 2620, 836: 6101}, // 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, 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}, - {459: 1967, 1967, 479: 1967, 484: 1967, 492: 1967, 553: 1967, 555: 1967, 632: 1967, 641: 1967, 653: 1967, 731: 1967}, + {459: 2498, 2497, 479: 6096, 484: 2496, 491: 2482, 553: 2495, 555: 2481, 632: 2491, 641: 2595, 653: 2611, 696: 2612, 731: 2465, 738: 2613, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 2619, 2618, 2468, 763: 2594, 2466, 770: 2616, 772: 2617, 2615, 783: 2467, 791: 2614, 809: 2620, 836: 6097}, + {459: 1970, 1970, 479: 1970, 484: 1970, 491: 1970, 553: 1970, 555: 1970, 632: 1970, 641: 1970, 653: 1970, 731: 1970}, + {459: 1969, 1969, 479: 1969, 484: 1969, 491: 1969, 553: 1969, 555: 1969, 632: 1969, 641: 1969, 653: 1969, 731: 1969}, + {459: 1968, 1968, 479: 1968, 484: 1968, 491: 1968, 553: 1968, 555: 1968, 632: 1968, 641: 1968, 653: 1968, 731: 1968}, + {459: 1967, 1967, 479: 1967, 484: 1967, 491: 1967, 553: 1967, 555: 1967, 632: 1967, 641: 1967, 653: 1967, 731: 1967}, // 3645 - {459: 1966, 1966, 479: 1966, 484: 1966, 492: 1966, 553: 1966, 555: 1966, 632: 1966, 641: 1966, 653: 1966, 731: 1966}, - {459: 1965, 1965, 479: 1965, 484: 1965, 492: 1965, 553: 1965, 555: 1965, 632: 1965, 641: 1965, 653: 1965, 731: 1965}, - {37: 6097}, + {459: 1966, 1966, 479: 1966, 484: 1966, 491: 1966, 553: 1966, 555: 1966, 632: 1966, 641: 1966, 653: 1966, 731: 1966}, + {459: 1965, 1965, 479: 1965, 484: 1965, 491: 1965, 553: 1965, 555: 1965, 632: 1965, 641: 1965, 653: 1965, 731: 1965}, + {37: 6098}, {1972, 1972}, - {493: 2638, 722: 6098}, + {493: 2639, 722: 6099}, // 3650 {1973, 1973}, - {37: 6101}, + {37: 6102}, {1974, 1974}, - {493: 2638, 722: 6102}, + {493: 2639, 722: 6103}, {1975, 1975}, // 3655 - {493: 2638, 722: 6104}, + {493: 2639, 722: 6105}, {1976, 1976}, {1978, 1978}, - {1984, 1984}, - {482: 6123}, + {1985, 1985}, + {482: 6126}, // 3660 - {449, 449, 466: 780, 476: 780, 780, 480: 2630, 488: 2631, 490: 2627, 753: 3775, 3776}, + {82: 2459, 152: 2461, 157: 2487, 161: 2458, 459: 2498, 2497, 484: 2496, 491: 2482, 498: 6112, 553: 2495, 555: 2481, 632: 2491, 641: 2595, 696: 6110, 731: 2465, 738: 6111, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 6118, 6117, 2468, 763: 2594, 2466, 770: 6115, 772: 6116, 6114, 783: 2467, 791: 6113, 804: 6124, 851: 6120, 855: 6121, 864: 6119, 870: 6122, 873: 6123, 952: 6125}, + {449, 449, 466: 780, 476: 780, 780, 480: 2631, 488: 2632, 490: 2628, 753: 3776, 3777}, {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, 959: 5514, 1135: 5507}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 5499, 5504, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 5502, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 5501, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 5506, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 5500, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 5507, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 5503, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 486: 3722, 554: 5513, 574: 5512, 634: 3720, 648: 5510, 2665, 2666, 2664, 759: 5514, 817: 5511, 960: 5515, 1135: 5508}, {456, 456}, - {455, 455}, // 3665 + {455, 455}, {454, 454}, {453, 453}, {452, 452}, {450, 450}, - {448, 448}, // 3670 + {448, 448}, {447, 447}, {446, 446}, {445, 445}, {444, 444}, - {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, 773: 6113, 775: 6114, 6112, 789: 2466, 795: 6111, 814: 6122, 873: 6118, 883: 6119, 888: 6117, 895: 6120, 898: 6121, 1128: 6125}, + {32: 5016}, {1983, 1983}, - {2010, 2010}, - {2009, 2009}, + {461: 6127}, + {82: 2459, 152: 2461, 157: 2487, 161: 2458, 459: 2498, 2497, 484: 2496, 491: 2482, 498: 6112, 553: 2495, 555: 2481, 632: 2491, 641: 2595, 696: 6110, 731: 2465, 738: 6111, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 6118, 6117, 2468, 763: 2594, 2466, 770: 6115, 772: 6116, 6114, 783: 2467, 791: 6113, 804: 6124, 851: 6120, 855: 6121, 864: 6119, 870: 6122, 873: 6123, 952: 6128}, + {1984, 1984}, // 3680 + {2011, 2011}, + {2010, 2010}, {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, 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}, + {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: 5219, 5218, 5217, 824: 5220, 869: 6133}, + {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, 6135, 1008, 1008, 1008, 1008, 1008, 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, 1087: 6134}, // 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, 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, 890: 6138}, - {7: 1012, 468: 1012, 487: 1012}, + {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: 4072, 487: 1803, 839: 6136}, + {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 487: 6137, 648: 6139, 2665, 2666, 2664, 902: 6140, 948: 6138}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 6154, 2665, 2666, 2664, 724: 6152, 902: 6140, 948: 6153}, + {7: 6148, 487: 6147}, // 3690 + {7: 1010, 468: 1010, 487: 1010, 639: 6142, 892: 6141}, + {7: 1012, 468: 1012, 487: 1012}, {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, 890: 6142}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 509: 6144, 648: 6143, 2665, 2666, 2664}, + {7: 1010, 468: 1010, 487: 1010, 639: 6146, 892: 6145}, + // 3695 {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, 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, 777: 2625, 6148}, + {509: 6144}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 5231, 565: 5226, 648: 3806, 2665, 2666, 2664, 696: 5230, 724: 5229, 785: 5228, 788: 5227, 5233, 835: 5223, 874: 6150}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 6139, 2665, 2666, 2664, 902: 6149}, // 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, 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}, + {7: 1011, 468: 1011, 487: 1011}, + {237, 237, 7: 5277, 468: 237, 489: 2625, 777: 2626, 6151}, + {2015, 2015, 468: 2015}, + {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, 5281, 480: 887, 485: 887, 489: 887, 887, 511: 887, 843: 6160}, + {7: 6148, 468: 6157}, // 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, 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, 900: 5235, 6158}, + {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: 6155, 892: 6141}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 509: 6144, 648: 6156, 2665, 2666, 2664}, + {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: 6146, 892: 6145}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 5231, 565: 5226, 648: 3806, 2665, 2666, 2664, 696: 5230, 724: 5229, 785: 5228, 788: 5227, 5233, 835: 5223, 874: 6158}, + {237, 237, 7: 5277, 489: 2625, 777: 2626, 6159}, // 3710 - {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, 1049: 6162}, - {2015, 2015, 468: 2015}, + {2014, 2014}, + {885, 885, 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 465: 5238, 468: 885, 474: 885, 480: 885, 485: 885, 489: 885, 885, 511: 885, 648: 5237, 2665, 2666, 2664, 900: 5236, 6161}, + {866, 866, 468: 866, 474: 5291, 480: 866, 485: 5292, 489: 866, 866, 511: 5290, 924: 5294, 5293, 1041: 5295, 6162}, + {237, 237, 468: 237, 480: 237, 489: 2625, 237, 777: 2626, 6163}, + {1258, 1258, 468: 1258, 480: 1258, 490: 2628, 753: 2629, 797: 6164}, // 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, 1124: 6222}, + {848, 848, 468: 848, 480: 5341, 1050: 6165}, + {2016, 2016, 468: 2016}, + {2017, 2017, 7: 3477}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 6240, 2665, 2666, 2664}, + {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: 4342, 765: 6238}, // 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, 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, 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}, + {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: 4342, 765: 6229}, + {110: 5355, 553: 5354, 1125: 6225}, + {151: 573, 158: 5404}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 552: 6220, 648: 3806, 2665, 2666, 2664, 724: 3807, 782: 6219}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 552: 6216, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5114, 848: 6215}, // 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, 782: 6178}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 5533, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 5538, 552: 6212, 648: 3378, 2665, 2666, 2664, 725: 5067, 787: 5540, 807: 5541, 5539, 845: 6211}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 6207}, + {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: 4342, 765: 6205}, + {151: 6185}, + {155: 6182}, // 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, 865: 6181}, - {53, 53}, - {479: 6183}, + {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: 4342, 765: 6180}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 3807, 782: 6181}, + {26, 26, 7: 3809}, + {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: 4342, 765: 6183}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4862, 2665, 2666, 2664, 868: 6184}, // 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, 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}, + {53, 53}, + {479: 6186}, + {459: 2498, 2497, 484: 2496, 491: 2482, 553: 2495, 555: 2481, 632: 2491, 641: 2595, 696: 6189, 731: 6187, 738: 6190, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 6192, 6191, 6188, 763: 2594, 6194, 770: 6195, 772: 6196, 6193, 878: 6197}, + {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, 746: 5214, 847: 5215, 903: 6200}, + {459: 2498, 484: 2496, 553: 2495, 632: 2491, 641: 2595, 696: 3773, 738: 3772, 2492, 2493, 2494, 2503, 2501, 3774, 3775, 763: 6131}, // 3740 + {175, 175, 466: 780, 468: 175, 476: 780, 780, 480: 2631, 488: 2632, 490: 2628, 753: 3776, 3777}, + {177, 177, 466: 781, 468: 177, 476: 781, 781}, {178, 178, 468: 178}, {176, 176, 468: 176}, {174, 174, 468: 174}, + // 3745 {173, 173, 468: 173}, {172, 172, 468: 172}, - // 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, 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, 866: 6198}, + {169, 169, 468: 6198}, + {459: 2498, 2497, 484: 2496, 491: 2482, 553: 2495, 555: 2481, 632: 2491, 641: 2595, 696: 6189, 731: 6187, 738: 6190, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 6192, 6191, 6188, 763: 2594, 6194, 770: 6195, 772: 6196, 6193, 878: 6199}, // 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, 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, 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}, + {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: 5219, 5218, 5217, 824: 5220, 869: 6201}, + {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, 6135, 1008, 1008, 1008, 1008, 1008, 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, 1087: 6202}, + {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: 4072, 487: 1803, 839: 6203}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 487: 6204, 648: 6139, 2665, 2666, 2664, 902: 6140, 948: 6138}, // 3755 + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 6152}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 6206, 2665, 2666, 2664}, {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, 774: 6207}, - {1993, 1993}, - {1994, 1994, 7: 4626}, + {1996, 1996, 162: 6209, 475: 6208}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4626, 2665, 2666, 2664, 776: 6210}, // 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, 785: 5539, 806: 5540, 5538, 844: 6211}, - {1996, 1996, 7: 5541}, - {1999, 1999, 7: 5115}, + {1994, 1994}, + {1995, 1995, 7: 4627}, + {1998, 1998, 7: 5542}, + {569: 6213}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 5533, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 5538, 648: 3378, 2665, 2666, 2664, 725: 5067, 787: 5540, 807: 5541, 5539, 845: 6214}, // 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, 894: 6221}, - {569: 6218}, + {1997, 1997, 7: 5542}, + {2000, 2000, 7: 5116}, + {569: 6217}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5114, 848: 6218}, + {1999, 1999, 7: 5116}, // 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, 782: 6219}, - {1992, 1992, 7: 3808, 657: 4677, 659: 4676, 894: 6220}, - {2000, 2000}, + {1993, 1993, 7: 3809, 657: 4678, 659: 4677, 896: 6224}, + {569: 6221}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 3807, 782: 6222}, + {1993, 1993, 7: 3809, 657: 4678, 659: 4677, 896: 6223}, {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, 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, 862: 6230, 1042: 6232}, - {2133, 2133, 92: 4125, 850: 6234}, - {2132, 2132, 483: 4126, 862: 6233}, + {2002, 2002}, + {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: 4342, 765: 6226}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 3807, 782: 6227}, + {1993, 1993, 7: 3809, 657: 4678, 659: 4677, 896: 6228}, {2006, 2006}, + // 3780 + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 6230, 2665, 2666, 2664}, + {458: 6231}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 6232}, + {2135, 2135, 92: 4126, 483: 4127, 850: 6234, 865: 6233, 1043: 6235}, + {2134, 2134, 92: 4126, 850: 6237}, // 3785 - {2130, 2130}, - {2131, 2131}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 6236}, + {2133, 2133, 483: 4127, 865: 6236}, {2007, 2007}, - {2142, 2142}, + {2131, 2131}, + {2132, 2132}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5212, 2665, 2666, 2664, 801: 6239}, // 3790 - {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: 6421}, - {635: 6409}, + {2008, 2008}, + {2143, 2143}, + {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: 4702, 769: 6424}, + {635: 6412}, + {635: 2129}, + // 3795 {635: 2128}, {635: 2127}, - {635: 2126}, - // 3795 - {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: 6391}, - {92: 6353, 99: 2033, 139: 2033, 655: 2033, 1306: 6352}, - {492: 6351}, - {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: 6339}, - {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, 461: 1805, 552: 4701, 556: 1805, 769: 6307}, + {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: 4702, 769: 6394}, + {16: 6348, 92: 6347, 99: 2034, 139: 2034, 655: 2034, 1306: 6346}, + {491: 6345}, // 3800 - {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, 461: 1805, 552: 4701, 769: 6301}, - {151: 6296}, - {154: 6288}, - {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}, + {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: 4702, 769: 6333}, + {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, 461: 1805, 552: 4702, 556: 1805, 769: 6301}, + {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, 461: 1805, 552: 4702, 769: 6295}, + {151: 6290}, + {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: 4702, 769: 6254}, // 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, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 6255}, + {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: 6263, 6260, 6266, 6267, 6268, 6261, 6259, 6269, 6265, 6262, 464: 49, 466: 49, 49, 485: 49, 49, 634: 49, 49, 644: 6264, 898: 6258, 1172: 6256, 1264: 6257}, + {385, 385, 4: 4132, 4134, 389, 13: 2107, 4151, 4078, 4090, 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 4149, 4169, 4153, 4140, 4133, 4136, 4135, 4138, 4139, 4141, 4148, 389, 4159, 4160, 4146, 4147, 4152, 4154, 4166, 4165, 4171, 4167, 4164, 4157, 4162, 4163, 4156, 4158, 4161, 4150, 464: 4131, 466: 4168, 2107, 485: 4845, 2107, 634: 2107, 4137, 758: 4088, 762: 4089, 766: 4142, 779: 4144, 798: 4143, 821: 4145, 825: 4155, 829: 4170, 904: 5426, 999: 6289}, + {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: 6263, 6260, 6266, 6267, 6268, 6261, 6259, 6269, 6265, 6262, 464: 48, 466: 48, 48, 485: 48, 48, 634: 48, 48, 644: 6264, 898: 6288}, {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 - {460: 6279, 471: 1987, 1987, 482: 4171, 493: 1987, 723: 6278}, - {471: 1987, 1987, 482: 4171, 493: 1987, 723: 6276}, + {471: 1988, 1988, 482: 4172, 493: 1988, 646: 6285, 723: 6284}, + {460: 6281, 471: 1988, 1988, 482: 4172, 493: 1988, 723: 6280}, + {471: 1988, 1988, 482: 4172, 493: 1988, 723: 6278}, {40, 40, 4: 40, 40, 40, 13: 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 81: 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 464: 40, 466: 40, 40, 485: 40, 40, 634: 40, 40, 644: 40}, - {83: 6274, 85: 6275, 6272, 644: 6273}, - {471: 1987, 1987, 482: 4171, 493: 1987, 723: 6270}, + {83: 6276, 85: 6277, 6274, 644: 6275}, // 3815 + {471: 1988, 1988, 482: 4172, 493: 1988, 723: 6272}, {37, 37, 4: 37, 37, 37, 13: 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 81: 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 464: 37, 466: 37, 37, 485: 37, 37, 634: 37, 37, 644: 37}, - {471: 1987, 1987, 482: 4171, 493: 1987, 723: 6268}, + {471: 1988, 1988, 482: 4172, 493: 1988, 723: 6270}, {34, 34, 4: 34, 34, 34, 13: 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 81: 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 464: 34, 466: 34, 34, 485: 34, 34, 634: 34, 34, 644: 34}, {32, 32, 4: 32, 32, 32, 13: 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 81: 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 464: 32, 466: 32, 32, 485: 32, 32, 634: 32, 32, 644: 32}, - {31, 31, 4: 31, 31, 31, 13: 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 81: 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 464: 31, 466: 31, 31, 485: 31, 31, 634: 31, 31, 644: 31}, // 3820 - {471: 3938, 3937, 493: 2638, 722: 3934, 748: 3936, 799: 6269}, + {31, 31, 4: 31, 31, 31, 13: 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 81: 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 464: 31, 466: 31, 31, 485: 31, 31, 634: 31, 31, 644: 31}, + {471: 3939, 3938, 493: 2639, 722: 3935, 748: 3937, 799: 6271}, {35, 35, 4: 35, 35, 35, 13: 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 81: 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 464: 35, 466: 35, 35, 485: 35, 35, 634: 35, 35, 644: 35}, - {471: 3938, 3937, 493: 2638, 722: 3934, 748: 3936, 799: 6271}, + {471: 3939, 3938, 493: 2639, 722: 3935, 748: 3937, 799: 6273}, {38, 38, 4: 38, 38, 38, 13: 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 81: 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 464: 38, 466: 38, 38, 485: 38, 38, 634: 38, 38, 644: 38}, - {39, 39, 4: 39, 39, 39, 13: 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 81: 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 464: 39, 466: 39, 39, 485: 39, 39, 634: 39, 39, 644: 39}, // 3825 + {39, 39, 4: 39, 39, 39, 13: 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 81: 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 464: 39, 466: 39, 39, 485: 39, 39, 634: 39, 39, 644: 39}, {36, 36, 4: 36, 36, 36, 13: 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 81: 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 464: 36, 466: 36, 36, 485: 36, 36, 634: 36, 36, 644: 36}, {33, 33, 4: 33, 33, 33, 13: 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 81: 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 464: 33, 466: 33, 33, 485: 33, 33, 634: 33, 33, 644: 33}, {30, 30, 4: 30, 30, 30, 13: 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 81: 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 464: 30, 466: 30, 30, 485: 30, 30, 634: 30, 30, 644: 30}, - {471: 3938, 3937, 493: 2638, 722: 3934, 748: 3936, 799: 6277}, - {41, 41, 4: 41, 41, 41, 13: 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 81: 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 464: 41, 466: 41, 41, 485: 41, 41, 634: 41, 41, 644: 41}, + {471: 3939, 3938, 493: 2639, 722: 3935, 748: 3937, 799: 6279}, // 3830 - {471: 3938, 3937, 493: 2638, 722: 3934, 748: 3936, 799: 6281}, - {471: 3938, 3937, 493: 2638, 722: 3934, 748: 3936, 799: 6280}, + {41, 41, 4: 41, 41, 41, 13: 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 81: 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 464: 41, 466: 41, 41, 485: 41, 41, 634: 41, 41, 644: 41}, + {471: 3939, 3938, 493: 2639, 722: 3935, 748: 3937, 799: 6283}, + {471: 3939, 3938, 493: 2639, 722: 3935, 748: 3937, 799: 6282}, {42, 42, 4: 42, 42, 42, 13: 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 81: 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 464: 42, 466: 42, 42, 485: 42, 42, 634: 42, 42, 644: 42}, {43, 43, 4: 43, 43, 43, 13: 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 81: 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 464: 43, 466: 43, 43, 485: 43, 43, 634: 43, 43, 644: 43}, - {471: 3938, 3937, 493: 2638, 722: 3934, 748: 3936, 799: 6285}, // 3835 - {471: 3938, 3937, 493: 2638, 722: 3934, 748: 3936, 799: 6284}, + {471: 3939, 3938, 493: 2639, 722: 3935, 748: 3937, 799: 6287}, + {471: 3939, 3938, 493: 2639, 722: 3935, 748: 3937, 799: 6286}, {44, 44, 4: 44, 44, 44, 13: 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 81: 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 464: 44, 466: 44, 44, 485: 44, 44, 634: 44, 44, 644: 44}, {45, 45, 4: 45, 45, 45, 13: 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 81: 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 464: 45, 466: 45, 45, 485: 45, 45, 634: 45, 45, 644: 45}, {46, 46, 4: 46, 46, 46, 13: 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 81: 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 464: 46, 466: 46, 46, 485: 46, 46, 634: 46, 46, 644: 46}, - {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, 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}, + {50, 50}, + {479: 6291}, + {459: 2498, 2497, 484: 2496, 491: 2482, 553: 2495, 555: 2481, 632: 2491, 641: 2595, 696: 6189, 731: 6187, 738: 6190, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 6192, 6191, 6188, 763: 2594, 6194, 770: 6195, 772: 6196, 6193, 878: 6292}, + {468: 6293}, + {459: 2498, 2497, 484: 2496, 491: 2482, 553: 2495, 555: 2481, 632: 2491, 641: 2595, 696: 6189, 731: 6187, 738: 6190, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 6192, 6191, 6188, 763: 2594, 6194, 770: 6195, 772: 6196, 6193, 878: 6294}, // 3845 - {2442, 2442, 7: 2442, 15: 2442, 2442, 2442, 2442, 2442, 2442, 2442, 2442, 2442, 2442, 2442}, - {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, 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, 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, 785: 5539, 806: 6303, 5538, 1099: 6304, 1259: 6302}, - {232, 232, 7: 6305}, - // 3855 + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 5533, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 5538, 648: 3378, 2665, 2666, 2664, 725: 5067, 787: 5540, 807: 6297, 5539, 1100: 6298, 1259: 6296}, + {232, 232, 7: 6299}, {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, 785: 5539, 806: 6303, 5538, 1099: 6306}, + // 3850 + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 5533, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 5538, 648: 3378, 2665, 2666, 2664, 725: 5067, 787: 5540, 807: 6297, 5539, 1100: 6300}, {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, 870: 5130, 906: 6308}, - // 3860 - {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 + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5130, 875: 5131, 906: 6302}, + {215, 215, 7: 5133, 14: 215, 58: 215, 460: 215, 652: 5177, 941: 5176, 6303}, + {223, 223, 14: 223, 58: 223, 460: 6305, 990: 6304}, + // 3855 + {202, 202, 14: 6322, 58: 6320, 934: 6321, 6319, 1080: 6318, 6317}, + {126: 6310, 6308, 6309, 6311, 989: 6307, 1170: 6306}, + {222, 222, 14: 222, 58: 222, 126: 6310, 6308, 6309, 6311, 989: 6316}, {221, 221, 14: 221, 58: 221, 126: 221, 221, 221, 221}, - {493: 2638, 722: 3934, 748: 6321}, - {493: 2638, 722: 3934, 748: 6320}, - {493: 2638, 722: 3934, 748: 6319}, - {493: 2638, 722: 3934, 748: 6318}, - // 3870 + {493: 2639, 722: 3935, 748: 6315}, + // 3860 + {493: 2639, 722: 3935, 748: 6314}, + {493: 2639, 722: 3935, 748: 6313}, + {493: 2639, 722: 3935, 748: 6312}, {216, 216, 14: 216, 58: 216, 126: 216, 216, 216, 216}, {217, 217, 14: 217, 58: 217, 126: 217, 217, 217, 217}, + // 3865 {218, 218, 14: 218, 58: 218, 126: 218, 218, 218, 218}, {219, 219, 14: 219, 58: 219, 126: 219, 219, 219, 219}, {220, 220, 14: 220, 58: 220, 126: 220, 220, 220, 220}, - // 3875 {233, 233}, - {201, 201, 14: 6328, 58: 6326, 934: 6327, 6338}, + {201, 201, 14: 6322, 58: 6320, 934: 6321, 6332}, + // 3870 {200, 200, 14: 200, 58: 200}, - {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, 1162: 6330}, + {483: 6331, 955: 6330}, + {196, 196, 14: 196, 58: 196, 202: 6326, 464: 6327, 567: 6325}, + {320: 6323}, + {191, 191, 14: 191, 58: 191, 202: 191, 464: 191, 567: 191, 1162: 6324}, + // 3875 {192, 192, 14: 192, 58: 192, 202: 192, 464: 192, 567: 192}, - {493: 2638, 722: 3934, 748: 6334}, + {493: 2639, 722: 3935, 748: 6328}, {194, 194, 14: 194, 58: 194}, - // 3885 {193, 193, 14: 193, 58: 193}, - {106: 6335}, + {106: 6329}, + // 3880 {195, 195, 14: 195, 58: 195}, {198, 198, 14: 198, 58: 198}, {197, 197, 14: 197, 58: 197}, - // 3890 {199, 199, 14: 199, 58: 199}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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, 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, 879: 6350}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 6334, 2665, 2666, 2664}, + // 3885 + {487: 6335}, + {461: 6336}, + {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: 6339, 458: 1900, 491: 6338, 640: 1900, 1023: 6337}, + {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, 883: 6344}, {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}, + // 3890 + {192: 6342, 374: 6343, 630: 6341, 638: 6340}, {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}, {1897, 1897, 27: 1897, 59: 1897, 61: 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 458: 1897, 640: 1897}, - // 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, 875: 6027, 6026, 6029, 6008, 880: 6028}, - {92: 2034, 99: 2034, 139: 2034, 655: 2034}, - {99: 2029, 139: 6359, 655: 2029, 1308: 6358}, + // 3895 + {1912, 1912, 27: 6035, 59: 6011, 61: 6031, 6024, 6014, 6010, 6018, 6022, 6034, 6017, 6023, 6021, 6019, 6032, 6025, 6013, 6033, 6012, 6015, 6016, 6020, 458: 6026, 640: 6036, 879: 6028, 6027, 6030, 6009, 884: 6029}, + {16: 2035, 92: 2035, 99: 2035, 139: 2035, 655: 2035}, + {99: 2030, 139: 6362, 655: 2030, 1308: 6361}, + {482: 6357}, + {155: 6349}, + // 3900 + {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: 4702, 769: 6350}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4862, 2665, 2666, 2664, 868: 6351}, + {15: 4078, 17: 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 758: 6352, 1082: 6353}, + {2444, 2444, 7: 2444, 15: 2444, 17: 2444, 2444, 2444, 2444, 2444, 2444, 2444, 2444, 2444, 2444}, + {52, 52, 7: 6355, 15: 4078, 17: 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 758: 6354}, // 3905 - {482: 6354}, - {342: 6356, 379: 6357, 389: 6355}, + {2443, 2443, 7: 2443, 15: 2443, 17: 2443, 2443, 2443, 2443, 2443, 2443, 2443, 2443, 2443, 2443}, + {15: 4078, 17: 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 758: 6356}, + {2442, 2442, 7: 2442, 15: 2442, 17: 2442, 2442, 2442, 2442, 2442, 2442, 2442, 2442, 2442, 2442}, + {342: 6359, 379: 6360, 389: 6358}, + {99: 2033, 139: 2033, 655: 2033}, + // 3910 {99: 2032, 139: 2032, 655: 2032}, {99: 2031, 139: 2031, 655: 2031}, - {99: 2030, 139: 2030, 655: 2030}, - // 3910 - {99: 2027, 655: 6363, 1311: 6362}, - {482: 6360}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 6361}, - {99: 2028, 655: 2028}, - {99: 6367}, + {99: 2028, 655: 6366, 1311: 6365}, + {482: 6363}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 6364}, // 3915 - {367: 6364}, - {139: 6365, 333: 6366}, - {99: 2026}, - {99: 2025}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 6369, 1310: 6368}, + {99: 2029, 655: 2029}, + {99: 6370}, + {367: 6367}, + {139: 6368, 333: 6369}, + {99: 2027}, // 3920 - {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, 1164: 6372}, - {7: 6375, 57: 6374}, + {99: 2026}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 6372, 1310: 6371}, + {459: 6374, 465: 2024, 1309: 6373}, + {459: 2025, 465: 2025}, + {465: 6380}, // 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, 1174: 6382}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 6376, 2665, 2666, 2664, 1164: 6375}, + {7: 6378, 57: 6377}, + {7: 2022, 57: 2022}, + {465: 2023}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 6379, 2665, 2666, 2664}, // 3930 - {2042, 2042, 460: 2042}, - {2041, 2041, 460: 2041, 466: 781, 476: 781, 781}, - {2040, 2040, 460: 2040}, - {2039, 2039, 460: 2039, 466: 780, 476: 780, 780, 480: 2630, 488: 2631, 490: 2627, 753: 3775, 3776}, - {2019, 2019, 460: 6384, 1307: 6383}, + {7: 2021, 57: 2021}, + {459: 2498, 2497, 484: 2496, 553: 2495, 632: 2491, 696: 6384, 738: 6382, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 6383, 6381, 3765, 1174: 6385}, + {2043, 2043, 460: 2043}, + {2042, 2042, 460: 2042, 466: 781, 476: 781, 781}, + {2041, 2041, 460: 2041}, // 3935 - {2036, 2036}, - {136: 6386, 302: 6385}, - {571: 6389}, - {571: 6387}, - {889: 6388}, + {2040, 2040, 460: 2040, 466: 780, 476: 780, 780, 480: 2631, 488: 2632, 490: 2628, 753: 3776, 3777}, + {2020, 2020, 460: 6387, 1307: 6386}, + {2037, 2037}, + {136: 6389, 302: 6388}, + {571: 6392}, // 3940 - {2017, 2017}, - {889: 6390}, + {571: 6390}, + {891: 6391}, {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, 779: 6394, 832: 6397, 916: 6396, 1175: 6393}, + {891: 6393}, + {2019, 2019}, // 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, 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}, - {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: 6407}, - {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: 6405}, - {461: 1987, 482: 4171, 723: 6403}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5212, 2665, 2666, 2664, 801: 6395}, + {2116, 2116, 13: 2107, 15: 4078, 2107, 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 28: 2107, 464: 4131, 467: 2107, 486: 2107, 634: 2107, 758: 4088, 762: 4089, 766: 6398, 779: 6397, 833: 6400, 916: 6399, 1175: 6396}, + {2124, 2124}, + {13: 3721, 16: 4090, 28: 6404, 467: 6403, 486: 3722, 634: 3720, 759: 6402, 762: 6405}, {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}, + // 3950 + {2115, 2115, 13: 2107, 15: 4078, 2107, 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 28: 2107, 464: 4131, 467: 2107, 486: 2107, 634: 2107, 758: 4088, 762: 4089, 766: 6398, 779: 6397, 833: 6401}, + {2114, 2114, 13: 2114, 15: 2114, 2114, 2114, 2114, 2114, 2114, 2114, 2114, 2114, 2114, 2114, 2114, 28: 2114, 464: 2114, 467: 2114, 486: 2114, 634: 2114}, + {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}, + {2: 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, 461: 1988, 482: 4172, 529: 1988, 723: 6410}, + {2: 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, 461: 1988, 482: 4172, 529: 1988, 723: 6408}, // 3955 - {461: 4189, 1020: 6404}, + {461: 1988, 482: 4172, 723: 6406}, {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}, + {461: 4190, 1021: 6407}, {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, 793: 6408}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 529: 3376, 648: 3378, 2665, 2666, 2664, 725: 3375, 853: 6409}, // 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}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 6411, 2664, 2665, 2663}, - {93: 4734, 458: 1788, 468: 4733, 841: 6413, 1208: 6412}, - {458: 6414}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 529: 3646, 648: 3378, 2665, 2666, 2664, 725: 3645, 794: 6411}, + {2121, 2121, 13: 2121, 15: 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 2121, 28: 2121, 464: 2121, 467: 2121, 486: 2121, 634: 2121}, + {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: 4702, 769: 6413}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 6414, 2665, 2666, 2664}, // 3965 + {93: 4735, 458: 1788, 468: 4734, 842: 6416, 1208: 6415}, + {458: 6417}, {458: 1787}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 6415}, - {459: 6416}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 6418}, + {459: 6419}, // 3970 - {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 459: 4508, 648: 4025, 2665, 2666, 2664, 729: 4507, 813: 4506, 822: 6420}, + {7: 4517, 57: 6421}, + {1799, 1799, 4: 1799, 29: 1799, 92: 1799, 1799, 1799, 1799, 1799, 1799, 460: 1799, 468: 1799, 483: 1799, 861: 6422}, + {2135, 2135, 4: 4731, 29: 4728, 92: 4126, 4735, 4595, 4320, 4596, 4319, 460: 4730, 468: 4734, 483: 4127, 840: 4732, 842: 4729, 849: 4733, 6234, 860: 4727, 865: 6233, 1043: 6423}, + {2142, 2142}, // 3975 - {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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 6425, 2665, 2666, 2664}, + {459: 6426}, + {221: 4764, 230: 4766, 233: 4765, 1117: 6427}, + {57: 6428}, + {458: 6429}, // 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, 794: 6429}, - {7: 4027, 57: 6430}, - {2143, 2143}, - {2235, 2235}, - {2260, 2260}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 6430}, + {459: 6431}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4025, 2665, 2666, 2664, 729: 4026, 795: 6432}, + {7: 4028, 57: 6433}, + {2144, 2144}, // 3985 - {2266, 2266, 460: 6435, 658: 6434}, - {149: 6442, 674: 6441}, - {303: 6437, 311: 6436}, - {61: 6440}, - {310: 6438}, + {2236, 2236}, + {2261, 2261}, + {2267, 2267, 460: 6438, 658: 6437}, + {149: 6445, 674: 6444}, + {303: 6440, 311: 6439}, // 3990 - {149: 6439}, - {2263, 2263}, + {61: 6443}, + {310: 6441}, + {149: 6442}, {2264, 2264}, {2265, 2265}, - {2262, 2262, 660: 5286, 908: 6443}, // 3995 - {2261, 2261}, + {2266, 2266}, + {2263, 2263, 660: 5287, 908: 6446}, + {2262, 2262}, + {2269, 2269}, {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, 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, 774: 6452}, - {7: 4626, 635: 6453}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 6462, 782: 6461}, + {553: 6451}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 6452}, + {475: 6454, 635: 6453}, + {873, 873, 2901, 2749, 2785, 2903, 2676, 873, 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 460: 873, 573: 4906, 648: 4905, 2665, 2666, 2664, 841: 6459}, // 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, 788: 6455}, - {2292, 2292}, - {2284, 2284, 7: 4907, 460: 4888, 788: 6457}, - {2295, 2295}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4626, 2665, 2666, 2664, 776: 6455}, + {7: 4627, 635: 6456}, + {873, 873, 2901, 2749, 2785, 2903, 2676, 873, 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 460: 873, 573: 4906, 648: 4905, 2665, 2666, 2664, 841: 6457}, + {2285, 2285, 7: 4908, 460: 4889, 790: 6458}, + {2293, 2293}, // 4010 - {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, 774: 6474}, - {244: 6470}, + {2285, 2285, 7: 4908, 460: 4889, 790: 6460}, + {2296, 2296}, + {2288, 2288, 7: 3809, 156: 6482, 460: 2288, 638: 6481, 963: 6492}, + {1016, 1016, 7: 1016, 98: 6467, 156: 1016, 460: 1016, 475: 6464, 635: 6463, 638: 1016, 641: 6465, 656: 6466}, + {873, 873, 2901, 2749, 2785, 2903, 2676, 873, 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 460: 873, 573: 4906, 648: 4905, 2665, 2666, 2664, 841: 6490}, // 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: 5656, 2664, 2665, 2663, 856: 6465}, - {2284, 2284, 7: 5658, 460: 4888, 788: 6466}, - {2289, 2289}, - {458: 6468}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4626, 2665, 2666, 2664, 776: 6477}, + {245: 6473}, + {245: 6470}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5657, 2665, 2666, 2664, 858: 6468}, + {2285, 2285, 7: 5659, 460: 4889, 790: 6469}, // 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: 5656, 2664, 2665, 2663, 856: 6469}, - {2290, 2290, 7: 5658}, + {2290, 2290}, {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: 5656, 2664, 2665, 2663, 856: 6472}, - {2284, 2284, 7: 5658, 460: 4888, 788: 6473}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5657, 2665, 2666, 2664, 858: 6472}, + {2291, 2291, 7: 5659}, + {458: 6474}, // 4025 - {2291, 2291}, - {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: 5656, 2664, 2665, 2663, 856: 6482}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5657, 2665, 2666, 2664, 858: 6475}, + {2285, 2285, 7: 5659, 460: 4889, 790: 6476}, + {2292, 2292}, + {2288, 2288, 7: 4627, 98: 6480, 156: 6482, 460: 2288, 635: 6479, 638: 6481, 963: 6478}, + {2285, 2285, 460: 4889, 790: 6489}, // 4030 - {98: 6481}, - {98: 6480}, - {2285, 2285, 460: 2285}, + {873, 873, 2901, 2749, 2785, 2903, 2676, 873, 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 460: 873, 573: 4906, 648: 4905, 2665, 2666, 2664, 841: 6487}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 5657, 2665, 2666, 2664, 858: 6485}, + {98: 6484}, + {98: 6483}, {2286, 2286, 460: 2286}, - {2284, 2284, 7: 5658, 460: 4888, 788: 6483}, // 4035 - {2288, 2288}, - {2284, 2284, 7: 4907, 460: 4888, 788: 6485}, - {2293, 2293}, + {2287, 2287, 460: 2287}, + {2285, 2285, 7: 5659, 460: 4889, 790: 6486}, + {2289, 2289}, + {2285, 2285, 7: 4908, 460: 4889, 790: 6488}, {2294, 2294}, - {2284, 2284, 7: 4907, 460: 4888, 788: 6488}, // 4040 - {2296, 2296}, - {2284, 2284, 460: 4888, 788: 6490}, + {2295, 2295}, + {2285, 2285, 7: 4908, 460: 4889, 790: 6491}, {2297, 2297}, - {553: 6496}, - {479: 6494}, + {2285, 2285, 460: 4889, 790: 6493}, + {2298, 2298}, // 4045 + {553: 6499}, + {479: 6497}, + {553: 2300}, + {475: 6498, 553: 2301}, {553: 2299}, - {475: 6495, 553: 2300}, - {553: 2298}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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, 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, 956: 6503}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 6500}, + {475: 5281, 541: 887, 635: 887, 646: 887, 843: 6501}, + {541: 6504, 635: 6503, 646: 6505, 1112: 6502}, + {2306, 2306}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 6512, 2665, 2666, 2664}, // 4055 - {2302, 2302, 7: 5793}, - {491: 6505}, - {459: 3781, 826: 6506}, - {15: 6507}, - {493: 2638, 722: 3934, 748: 6508}, + {459: 3782, 826: 6507}, + {459: 3782, 826: 5793, 957: 6506}, + {2303, 2303, 7: 5794}, + {492: 6508}, + {459: 3782, 826: 6509}, // 4060 - {2303, 2303}, - {541: 6501, 646: 6502, 1111: 6510}, + {15: 6510}, + {493: 2639, 722: 3935, 748: 6511}, {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, 1191: 6513}, + {541: 6504, 646: 6505, 1112: 6513}, + {2305, 2305}, // 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}, - {2306, 2306}, - {2: 2900, 2748, 2784, 2902, 2675, 8: 2721, 2676, 2807, 2919, 2912, 3255, 3260, 3033, 3110, 3114, 3103, 3113, 3115, 3106, 3111, 3112, 3116, 3109, 3062, 2787, 2707, 2789, 2763, 2710, 2699, 2732, 2791, 2792, 2896, 2786, 2920, 3022, 3021, 2674, 2785, 2788, 2799, 2739, 2743, 2795, 2905, 2754, 2833, 2672, 2673, 2832, 2904, 2671, 2917, 58: 2877, 2988, 2753, 2756, 2971, 2968, 2960, 2972, 2975, 2976, 2973, 2977, 2978, 2974, 2967, 2979, 2962, 2963, 2966, 2969, 2970, 2980, 3263, 2819, 2757, 2947, 2946, 2948, 2943, 2942, 2949, 2944, 2945, 2749, 2862, 2932, 2995, 2930, 2996, 2931, 2690, 2822, 2761, 3253, 2684, 2827, 2918, 3264, 3257, 2719, 3276, 2929, 2762, 3259, 3274, 3275, 3273, 3269, 2921, 2922, 2923, 2924, 2925, 2926, 2928, 3265, 2847, 2758, 2851, 2852, 2853, 2854, 2843, 2871, 2914, 2873, 2692, 2872, 2734, 2993, 2824, 2863, 2729, 2782, 2938, 2844, 2803, 2698, 2709, 2724, 2933, 2806, 2773, 2823, 2708, 3091, 2982, 3065, 2859, 2771, 3267, 2693, 2728, 3252, 2738, 2742, 2750, 2751, 2772, 2983, 2679, 2683, 2701, 3256, 2722, 2800, 2801, 2952, 2880, 2989, 2990, 2954, 2818, 2991, 2910, 3061, 3016, 2950, 2850, 3261, 2908, 2810, 2669, 2815, 2705, 2706, 2816, 2713, 2723, 2726, 2714, 2936, 2961, 2776, 2875, 2842, 2813, 2870, 2913, 2802, 2752, 3017, 2760, 3026, 3262, 2909, 2998, 2958, 2820, 2881, 2682, 2999, 3002, 2688, 2984, 3003, 3272, 2694, 2695, 2883, 3044, 3005, 2879, 2703, 3007, 2892, 2916, 2903, 2704, 3009, 2911, 2717, 2941, 3098, 2727, 2730, 2893, 2939, 3053, 3054, 2887, 3011, 3010, 2937, 2994, 2825, 3277, 3012, 3013, 2829, 2885, 3063, 3014, 2992, 2746, 2747, 2858, 2964, 2860, 3066, 3015, 2906, 2907, 2848, 2755, 2889, 3029, 3018, 2670, 3075, 2888, 3081, 3082, 3083, 3084, 3086, 3085, 3087, 3088, 3028, 2768, 2666, 2667, 2940, 2957, 2677, 2959, 2985, 2680, 2681, 3042, 3000, 3001, 2685, 2869, 2686, 2687, 2856, 3268, 3004, 2804, 2691, 2696, 2697, 3006, 3008, 3048, 3049, 2711, 2712, 2826, 2716, 2876, 3092, 2718, 2886, 3258, 2821, 2797, 2894, 2915, 2878, 2812, 2934, 3055, 2864, 2882, 2927, 2735, 2733, 2809, 2895, 2790, 2951, 2865, 2793, 2794, 3278, 2828, 2737, 2759, 3030, 3093, 2740, 2898, 2901, 2953, 2987, 3031, 2997, 2838, 2839, 2845, 3059, 3034, 3060, 2935, 3035, 2965, 2868, 2808, 2899, 2857, 3023, 3020, 3019, 3067, 2884, 2986, 2897, 3025, 2866, 2764, 2765, 3027, 3101, 3089, 2890, 2769, 2798, 2805, 2867, 3107, 2774, 3032, 2874, 3036, 2779, 3037, 3038, 3254, 3039, 3040, 3041, 3094, 3043, 3045, 3046, 3047, 2715, 2861, 3095, 2831, 3050, 2720, 3102, 3281, 3052, 3285, 3284, 3279, 3104, 3105, 3057, 3056, 2736, 3058, 3064, 2837, 2744, 2745, 2981, 2855, 3270, 3271, 3280, 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: 6517, 648: 3805, 2664, 2665, 2663, 724: 6518}, - {249: 6520}, - // 4070 - {2310, 2310, 493: 2638, 722: 3934, 748: 6519}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 6515}, + {2308, 2308, 637: 6517, 1191: 6516}, {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, 1126: 6532, 1296: 6531}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 6518, 2665, 2666, 2664}, + {2307, 2307}, + // 4070 + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 646: 6520, 648: 3806, 2665, 2666, 2664, 724: 6521}, + {250: 6523}, + {2311, 2311, 493: 2639, 722: 3935, 748: 6522}, + {2310, 2310}, + {493: 2639, 722: 3935, 748: 6524}, // 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, 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}, + {2312, 2312}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 6536, 1127: 6535, 1296: 6534}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 6529, 1132: 6528, 1301: 6527}, + {2316, 2316, 7: 6532}, + {2315, 2315, 7: 2315}, // 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, 1132: 6530}, + {637: 6530}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 6531}, {2313, 2313, 7: 2313}, - {2319, 2319, 7: 6536}, - {2318, 2318, 7: 2318}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 6529, 1132: 6533}, + {2314, 2314, 7: 2314}, // 4085 - {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, 1126: 6537}, + {2320, 2320, 7: 6539}, + {2319, 2319, 7: 2319}, + {637: 6537}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 6538}, {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, 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}, - {154: 6555}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 6536, 1127: 6540}, + {2318, 2318, 7: 2318}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 6590, 2107, 6595, 6597, 6591, 6596, 6599, 6593, 6589, 6594, 6598, 6592, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 464: 4131, 467: 2107, 486: 2107, 634: 2107, 648: 5212, 2665, 2666, 2664, 758: 4088, 762: 4089, 766: 6398, 779: 6397, 801: 6601, 833: 6400, 916: 6602}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 6580, 2665, 2666, 2664}, + {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: 4342, 556: 1807, 765: 6569}, // 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, 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}, + {264: 6563, 1210: 6562}, + {155: 6558}, + {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: 4342, 765: 6547}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 3806, 2665, 2666, 2664, 724: 6548}, + {81: 6263, 6260, 6266, 6267, 6268, 6261, 6259, 6269, 6265, 6262, 6552, 644: 6264, 898: 6551, 969: 6550, 1145: 6549}, // 4100 + {25, 25, 81: 6263, 6260, 6266, 6267, 6268, 6261, 6259, 6269, 6265, 6262, 6552, 644: 6264, 898: 6551, 969: 6557}, + {24, 24, 81: 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 644: 24}, {22, 22, 81: 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 644: 22}, - {21, 21, 81: 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 460: 6551, 471: 1987, 1987, 482: 4171, 493: 1987, 644: 21, 723: 6550}, - {471: 3938, 3937, 493: 2638, 722: 3934, 748: 3936, 799: 6553}, - {471: 3938, 3937, 493: 2638, 722: 3934, 748: 3936, 799: 6552}, - {19, 19, 81: 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 644: 19}, + {21, 21, 81: 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 460: 6554, 471: 1988, 1988, 482: 4172, 493: 1988, 644: 21, 723: 6553}, + {471: 3939, 3938, 493: 2639, 722: 3935, 748: 3937, 799: 6556}, // 4105 + {471: 3939, 3938, 493: 2639, 722: 3935, 748: 3937, 799: 6555}, + {19, 19, 81: 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 644: 19}, {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, 865: 6557}, - {15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 758: 6291, 1081: 6558}, + {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: 4342, 765: 6559}, // 4110 - {51, 51, 7: 6294, 15: 4077, 4082, 4084, 4078, 4083, 4086, 4080, 4076, 4081, 4085, 4079, 758: 6293}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 3268, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 648: 4862, 2665, 2666, 2664, 868: 6560}, + {15: 4078, 17: 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 758: 6352, 1082: 6561}, + {51, 51, 7: 6355, 15: 4078, 17: 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 758: 6354}, {229, 229}, - {383: 6561}, - {228, 228, 81: 6562}, - {165: 6563}, + {383: 6564}, // 4115 - {458: 6564}, - {195: 6565}, + {228, 228, 81: 6565}, + {157: 6566}, + {458: 6567}, + {195: 6568}, {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, 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, 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}, + {2: 2901, 2749, 2785, 2903, 2676, 8: 2722, 2677, 2808, 2920, 2913, 3256, 3261, 3034, 3063, 3111, 3115, 3104, 3114, 3116, 3107, 3112, 3113, 3117, 3110, 2788, 2708, 2790, 2764, 2711, 2700, 2733, 2792, 2793, 2897, 2787, 2921, 3023, 3022, 2675, 2786, 2789, 2800, 2740, 2744, 2796, 2906, 2755, 2834, 2673, 2674, 2833, 2905, 2672, 2918, 58: 2878, 2989, 2754, 2757, 2972, 2969, 2961, 2973, 2976, 2977, 2974, 2978, 2979, 2975, 2968, 2980, 2963, 2964, 2967, 2970, 2971, 2981, 3264, 2820, 2758, 2948, 2947, 2949, 2944, 2943, 2950, 2945, 2946, 2750, 2863, 2933, 2996, 2931, 2997, 2932, 2691, 2823, 2762, 3254, 2685, 2828, 2919, 3265, 3258, 2720, 3277, 2930, 2763, 3260, 3275, 3276, 3274, 3270, 2922, 2923, 2924, 2925, 2926, 2927, 2929, 3266, 2848, 2759, 2852, 2853, 2854, 2855, 2844, 2872, 2915, 2874, 2693, 2873, 2735, 2994, 2825, 2864, 2730, 2783, 2939, 2845, 2804, 2699, 2710, 2725, 2934, 2807, 2774, 2824, 2694, 2709, 3092, 2983, 3066, 2752, 2860, 2772, 6570, 2680, 2729, 3253, 2739, 2743, 2751, 2773, 2984, 2684, 2702, 3257, 2723, 2801, 2802, 2953, 2881, 2990, 2991, 2955, 2819, 2992, 2911, 3062, 3017, 2951, 2851, 3262, 2909, 2811, 2670, 2816, 2706, 2707, 2817, 2714, 2724, 2727, 2715, 2937, 2962, 2777, 2876, 3064, 2843, 2814, 2871, 2914, 2803, 2753, 3018, 2761, 3027, 3263, 2910, 2999, 2959, 2821, 2882, 2683, 3000, 3003, 2689, 2985, 3004, 3273, 2695, 2696, 2884, 3045, 3006, 2880, 2704, 3008, 2893, 2917, 2904, 2705, 3010, 2912, 2718, 2942, 3099, 2728, 2731, 2894, 2940, 3054, 3055, 2888, 3012, 3011, 2938, 2995, 2826, 3278, 3013, 3014, 2830, 2886, 3015, 2993, 2747, 2748, 2859, 2965, 2861, 3067, 3016, 2907, 2908, 2849, 2756, 2890, 3030, 3019, 2671, 3076, 2889, 3082, 3083, 3084, 3085, 3087, 3086, 3088, 3089, 3029, 2769, 2667, 2668, 2941, 2958, 2678, 2960, 2986, 2681, 2682, 3043, 3001, 3002, 2686, 2870, 2687, 2688, 2857, 3269, 3005, 2805, 2692, 2697, 2698, 3007, 3009, 3049, 3050, 2712, 2713, 2827, 2717, 2877, 3093, 2719, 2887, 3259, 2822, 2798, 2895, 2916, 2879, 2813, 2935, 3056, 2865, 2883, 2928, 2736, 2734, 2810, 2896, 2791, 2952, 2866, 2794, 2795, 3279, 2829, 2738, 2760, 3031, 3094, 2741, 2899, 2902, 2954, 2988, 3032, 2998, 2839, 2840, 2846, 3060, 3035, 3061, 2936, 3036, 2966, 2869, 2809, 2900, 2858, 3024, 3021, 3020, 3068, 2885, 2987, 2898, 3026, 2867, 2765, 2766, 3028, 3102, 3090, 2891, 2770, 2799, 2806, 2868, 3108, 2775, 3033, 2875, 3037, 2780, 3038, 3039, 3255, 3040, 3041, 3042, 3095, 3044, 3046, 3047, 3048, 2716, 2862, 3096, 2832, 3051, 2721, 3103, 3282, 3053, 3286, 3285, 3280, 3105, 3106, 3058, 3057, 2737, 3059, 3065, 2838, 2745, 2746, 2982, 2856, 3271, 3272, 3281, 2850, 2781, 2892, 2812, 2815, 3097, 3072, 3073, 3074, 3075, 3098, 3069, 3070, 3071, 2831, 3025, 3283, 3284, 3091, 3077, 3078, 3079, 3109, 3267, 461: 3377, 556: 5113, 648: 3378, 2665, 2666, 2664, 725: 5112, 760: 5130, 875: 5131, 906: 6571}, + {1660, 1660, 7: 1660, 14: 1660, 58: 1660, 141: 1660, 459: 6575, 1660, 554: 1660, 652: 1660, 654: 1660}, + {215, 215, 7: 5133, 14: 215, 58: 215, 460: 215, 652: 5177, 941: 5176, 6572}, + {223, 223, 14: 223, 58: 223, 460: 6305, 990: 6573}, + {202, 202, 14: 6322, 58: 6320, 934: 6321, 6319, 1080: 6318, 6574}, // 4125 - {141: 6574}, - {646: 6575}, - {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, 1022: 6578}, + {231, 231}, + {57: 6576}, + {141: 6577}, + {646: 6578}, + {461: 5146, 877: 6579}, // 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, 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}, + {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: 6339, 458: 1900, 491: 6338, 640: 1900, 1023: 6581}, + {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, 883: 6582}, + {1894, 1894, 27: 6035, 59: 6011, 61: 6031, 6024, 6014, 6010, 6018, 6022, 6034, 6017, 6023, 6021, 6019, 6032, 6025, 6013, 6033, 6012, 6015, 6016, 6020, 6584, 458: 6026, 640: 6036, 879: 6028, 6027, 6030, 6009, 884: 6029, 1204: 6583}, {1909, 1909}, - {198: 6583, 638: 6582}, - {545, 545, 553: 5981, 950: 6585}, // 4135 - {545, 545, 553: 5981, 950: 6584}, + {198: 6586, 638: 6585}, + {545, 545, 553: 5982, 950: 6588}, + {545, 545, 553: 5982, 950: 6587}, {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}, - {13: 1406, 15: 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 28: 1406, 461: 1987, 464: 1406, 467: 1406, 482: 4171, 486: 1406, 634: 1406, 723: 4880}, // 4140 - {13: 1336, 15: 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 28: 1336, 464: 1336, 467: 1336, 482: 4171, 486: 1336, 493: 1987, 634: 1336, 723: 4878}, - {13: 1330, 15: 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 28: 1330, 464: 1330, 467: 1330, 482: 4171, 486: 1330, 493: 1987, 634: 1330, 723: 4876}, - {13: 1333, 15: 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 28: 1333, 464: 1333, 467: 1333, 482: 4171, 486: 1333, 493: 1987, 634: 1333, 723: 4874}, - {13: 1327, 15: 1327, 1327, 1327, 1327, 1327, 1327, 1327, 1327, 1327, 1327, 1327, 1327, 28: 1327, 461: 1987, 464: 1327, 467: 1327, 482: 4171, 486: 1327, 634: 1327, 723: 4872}, - {13: 1329, 15: 1329, 1329, 1329, 1329, 1329, 1329, 1329, 1329, 1329, 1329, 1329, 1329, 28: 1329, 461: 1987, 464: 1329, 467: 1329, 482: 4171, 486: 1329, 634: 1329, 723: 4870}, + {13: 1328, 15: 1328, 1328, 1328, 1328, 1328, 1328, 1328, 1328, 1328, 1328, 1328, 1328, 28: 1328, 461: 1988, 464: 1328, 467: 1328, 482: 4172, 486: 1328, 634: 1328, 723: 4883}, + {13: 1406, 15: 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 28: 1406, 461: 1988, 464: 1406, 467: 1406, 482: 4172, 486: 1406, 634: 1406, 723: 4881}, + {13: 1336, 15: 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 28: 1336, 464: 1336, 467: 1336, 482: 4172, 486: 1336, 493: 1988, 634: 1336, 723: 4879}, + {13: 1330, 15: 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 28: 1330, 464: 1330, 467: 1330, 482: 4172, 486: 1330, 493: 1988, 634: 1330, 723: 4877}, + {13: 1333, 15: 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 28: 1333, 464: 1333, 467: 1333, 482: 4172, 486: 1333, 493: 1988, 634: 1333, 723: 4875}, // 4145 - {13: 1326, 15: 1326, 1326, 1326, 1326, 1326, 1326, 1326, 1326, 1326, 1326, 1326, 1326, 28: 1326, 461: 1987, 464: 1326, 467: 1326, 482: 4171, 486: 1326, 634: 1326, 723: 4868}, - {13: 1325, 15: 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 28: 1325, 461: 1987, 464: 1325, 467: 1325, 482: 4171, 486: 1325, 634: 1325, 723: 4866}, - {13: 1323, 15: 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 28: 1323, 461: 1987, 464: 1323, 467: 1323, 482: 4171, 486: 1323, 634: 1323, 723: 4864}, - {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}, + {13: 1327, 15: 1327, 1327, 1327, 1327, 1327, 1327, 1327, 1327, 1327, 1327, 1327, 1327, 28: 1327, 461: 1988, 464: 1327, 467: 1327, 482: 4172, 486: 1327, 634: 1327, 723: 4873}, + {13: 1329, 15: 1329, 1329, 1329, 1329, 1329, 1329, 1329, 1329, 1329, 1329, 1329, 1329, 28: 1329, 461: 1988, 464: 1329, 467: 1329, 482: 4172, 486: 1329, 634: 1329, 723: 4871}, + {13: 1326, 15: 1326, 1326, 1326, 1326, 1326, 1326, 1326, 1326, 1326, 1326, 1326, 1326, 28: 1326, 461: 1988, 464: 1326, 467: 1326, 482: 4172, 486: 1326, 634: 1326, 723: 4869}, + {13: 1325, 15: 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 1325, 28: 1325, 461: 1988, 464: 1325, 467: 1325, 482: 4172, 486: 1325, 634: 1325, 723: 4867}, + {13: 1323, 15: 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 28: 1323, 461: 1988, 464: 1323, 467: 1323, 482: 4172, 486: 1323, 634: 1323, 723: 4865}, // 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, 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}, + {13: 1324, 15: 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 1324, 28: 1324, 461: 1988, 464: 1324, 467: 1324, 482: 4172, 486: 1324, 634: 1324, 723: 4863}, + {13: 1377, 15: 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 28: 1377, 155: 4855, 464: 1377, 467: 1377, 486: 1377, 634: 1377}, + {13: 2107, 15: 4078, 2107, 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 28: 2107, 464: 4131, 467: 2107, 486: 2107, 634: 2107, 758: 4088, 762: 4089, 766: 6398, 779: 6397, 833: 6400, 916: 6603}, + {2125, 2125, 13: 2107, 15: 4078, 2107, 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 28: 2107, 464: 4131, 467: 2107, 486: 2107, 634: 2107, 758: 4088, 762: 4089, 766: 6398, 779: 6397, 833: 6401}, + {2126, 2126, 13: 2107, 15: 4078, 2107, 4083, 4085, 4079, 4084, 4087, 4081, 4077, 4082, 4086, 4080, 28: 2107, 464: 4131, 467: 2107, 486: 2107, 634: 2107, 758: 4088, 762: 4089, 766: 6398, 779: 6397, 833: 6401}, + // 4155 + {1986, 1986, 59: 2478, 80: 2593, 82: 2459, 91: 2489, 152: 2461, 157: 2487, 161: 2458, 165: 2483, 196: 2508, 203: 2605, 206: 2454, 215: 2507, 2474, 2460, 232: 2486, 237: 2464, 240: 2484, 242: 2455, 244: 2490, 261: 2476, 265: 2475, 272: 2488, 274: 2456, 277: 2477, 288: 2469, 459: 2498, 2497, 483: 2601, 2496, 491: 2482, 498: 2506, 511: 2596, 515: 2472, 553: 2495, 555: 2481, 632: 2491, 635: 2604, 640: 2457, 2595, 653: 2452, 656: 2463, 661: 2462, 666: 2505, 673: 2453, 696: 2502, 731: 2465, 738: 2504, 2492, 2493, 2494, 2503, 2501, 2500, 2499, 749: 2575, 2574, 2468, 763: 2594, 2466, 770: 2558, 772: 2569, 2585, 783: 2467, 791: 2524, 804: 2599, 809: 2512, 838: 2597, 846: 2479, 851: 2519, 855: 2522, 864: 2561, 870: 2566, 873: 2576, 915: 2531, 919: 2470, 955: 2600, 962: 2510, 964: 2511, 2514, 2515, 968: 2517, 970: 2516, 972: 2513, 974: 2518, 2520, 2521, 978: 2480, 2557, 981: 2527, 991: 2535, 2528, 2529, 2530, 2536, 2534, 2537, 2538, 1000: 2533, 2532, 1003: 2523, 2485, 2471, 2539, 2551, 2540, 2541, 2542, 2544, 2548, 2545, 2549, 2550, 2543, 2547, 2546, 1020: 2509, 1024: 2525, 2526, 2473, 1030: 2553, 2552, 1034: 2555, 2556, 2554, 1039: 2591, 2559, 1047: 2603, 2602, 2560, 1054: 2562, 1056: 2588, 1083: 2563, 2564, 1086: 2565, 1088: 2570, 1091: 2567, 2568, 1094: 2590, 2571, 2598, 2573, 2572, 1104: 2578, 2577, 2581, 1108: 2582, 1110: 2589, 1113: 2579, 6605, 1118: 2580, 1129: 2583, 2584, 2587, 1133: 2586}, {433, 433}, } ) @@ -10997,6 +11001,7 @@ type yyLexer interface { Lex(lval *yySymType) int Errorf(format string, a ...interface{}) error AppendError(err error) + AppendWarn(err error) Errors() (warns []error, errs []error) } @@ -13739,8 +13744,9 @@ yynewstate: case 464: { parser.yyVAL.statement = &ast.TraceStmt{ - Stmt: yyS[yypt-0].statement, - Format: "row", + Stmt: yyS[yypt-0].statement, + Format: "row", + TracePlan: false, } startOffset := parser.startOffset(&yyS[yypt]) yyS[yypt-0].statement.SetText(string(parser.src[startOffset:])) @@ -13748,13 +13754,23 @@ yynewstate: case 465: { parser.yyVAL.statement = &ast.TraceStmt{ - Stmt: yyS[yypt-0].statement, - Format: yyS[yypt-1].ident, + Stmt: yyS[yypt-0].statement, + Format: yyS[yypt-1].ident, + TracePlan: false, + } + startOffset := parser.startOffset(&yyS[yypt]) + yyS[yypt-0].statement.SetText(string(parser.src[startOffset:])) + } + case 466: + { + parser.yyVAL.statement = &ast.TraceStmt{ + Stmt: yyS[yypt-0].statement, + TracePlan: true, } startOffset := parser.startOffset(&yyS[yypt]) yyS[yypt-0].statement.SetText(string(parser.src[startOffset:])) } - case 469: + case 470: { parser.yyVAL.statement = &ast.ExplainStmt{ Stmt: &ast.ShowStmt{ @@ -13763,7 +13779,7 @@ yynewstate: }, } } - case 470: + case 471: { parser.yyVAL.statement = &ast.ExplainStmt{ Stmt: &ast.ShowStmt{ @@ -13773,49 +13789,49 @@ yynewstate: }, } } - case 471: + case 472: { parser.yyVAL.statement = &ast.ExplainStmt{ Stmt: yyS[yypt-0].statement, Format: "row", } } - case 472: + case 473: { parser.yyVAL.statement = &ast.ExplainForStmt{ Format: "row", ConnectionID: getUint64FromNUM(yyS[yypt-0].item), } } - case 473: + case 474: { parser.yyVAL.statement = &ast.ExplainForStmt{ Format: yyS[yypt-3].ident, ConnectionID: getUint64FromNUM(yyS[yypt-0].item), } } - case 474: + case 475: { parser.yyVAL.statement = &ast.ExplainStmt{ Stmt: yyS[yypt-0].statement, Format: yyS[yypt-1].ident, } } - case 475: + case 476: { parser.yyVAL.statement = &ast.ExplainForStmt{ Format: yyS[yypt-3].ident, ConnectionID: getUint64FromNUM(yyS[yypt-0].item), } } - case 476: + case 477: { parser.yyVAL.statement = &ast.ExplainStmt{ Stmt: yyS[yypt-0].statement, Format: yyS[yypt-1].ident, } } - case 477: + case 478: { parser.yyVAL.statement = &ast.ExplainStmt{ Stmt: yyS[yypt-0].statement, @@ -13823,7 +13839,7 @@ yynewstate: Analyze: true, } } - case 484: + case 485: { stmt := yyS[yypt-3].item.(*ast.BRIEStmt) stmt.Kind = ast.BRIEKindBackup @@ -13831,7 +13847,7 @@ yynewstate: stmt.Options = yyS[yypt-0].item.([]*ast.BRIEOption) parser.yyVAL.statement = stmt } - case 485: + case 486: { stmt := yyS[yypt-3].item.(*ast.BRIEStmt) stmt.Kind = ast.BRIEKindRestore @@ -13839,110 +13855,110 @@ yynewstate: stmt.Options = yyS[yypt-0].item.([]*ast.BRIEOption) parser.yyVAL.statement = stmt } - case 486: + case 487: { parser.yyVAL.item = &ast.BRIEStmt{} } - case 487: + case 488: { parser.yyVAL.item = &ast.BRIEStmt{Schemas: yyS[yypt-0].item.([]string)} } - case 488: + case 489: { parser.yyVAL.item = &ast.BRIEStmt{Tables: yyS[yypt-0].item.([]*ast.TableName)} } - case 489: + case 490: { parser.yyVAL.item = []string{yyS[yypt-0].ident} } - case 490: + case 491: { parser.yyVAL.item = append(yyS[yypt-2].item.([]string), yyS[yypt-0].ident) } - case 491: + case 492: { parser.yyVAL.item = []*ast.BRIEOption{} } - case 492: + case 493: { parser.yyVAL.item = append(yyS[yypt-1].item.([]*ast.BRIEOption), yyS[yypt-0].item.(*ast.BRIEOption)) } - case 493: + case 494: { parser.yyVAL.item = ast.BRIEOptionConcurrency } - case 494: + case 495: { parser.yyVAL.item = ast.BRIEOptionResume } - case 495: + case 496: { parser.yyVAL.item = ast.BRIEOptionSendCreds } - case 496: + case 497: { parser.yyVAL.item = ast.BRIEOptionOnline } - case 497: + case 498: { parser.yyVAL.item = ast.BRIEOptionCheckpoint } - case 498: + case 499: { parser.yyVAL.item = ast.BRIEOptionSkipSchemaFiles } - case 499: + case 500: { parser.yyVAL.item = ast.BRIEOptionStrictFormat } - case 500: + case 501: { parser.yyVAL.item = ast.BRIEOptionCSVNotNull } - case 501: + case 502: { parser.yyVAL.item = ast.BRIEOptionCSVBackslashEscape } - case 502: + case 503: { parser.yyVAL.item = ast.BRIEOptionCSVTrimLastSeparators } - case 503: + case 504: { parser.yyVAL.item = ast.BRIEOptionTiKVImporter } - case 504: + case 505: { parser.yyVAL.item = ast.BRIEOptionCSVSeparator } - case 505: + case 506: { parser.yyVAL.item = ast.BRIEOptionCSVDelimiter } - case 506: + case 507: { parser.yyVAL.item = ast.BRIEOptionCSVNull } - case 507: + case 508: { parser.yyVAL.item = ast.BRIEOptionBackend } - case 508: + case 509: { parser.yyVAL.item = ast.BRIEOptionOnDuplicate } - case 509: + case 510: { parser.yyVAL.item = ast.BRIEOptionOnDuplicate } - case 510: + case 511: { parser.yyVAL.item = &ast.BRIEOption{ Tp: yyS[yypt-2].item.(ast.BRIEOptionType), UintValue: yyS[yypt-0].item.(uint64), } } - case 511: + case 512: { value := uint64(0) if yyS[yypt-0].item.(bool) { @@ -13953,21 +13969,21 @@ yynewstate: UintValue: value, } } - case 512: + case 513: { parser.yyVAL.item = &ast.BRIEOption{ Tp: yyS[yypt-2].item.(ast.BRIEOptionType), StrValue: yyS[yypt-0].ident, } } - case 513: + case 514: { parser.yyVAL.item = &ast.BRIEOption{ Tp: yyS[yypt-2].item.(ast.BRIEOptionType), StrValue: strings.ToLower(yyS[yypt-0].ident), } } - case 514: + case 515: { unit, err := yyS[yypt-1].item.(ast.TimeUnitType).Duration() if err != nil { @@ -13980,35 +13996,35 @@ yynewstate: UintValue: yyS[yypt-2].item.(uint64) * uint64(unit), } } - case 515: + case 516: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionBackupTS, StrValue: yyS[yypt-0].ident, } } - case 516: + case 517: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionBackupTSO, UintValue: yyS[yypt-0].item.(uint64), } } - case 517: + case 518: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionLastBackupTS, StrValue: yyS[yypt-0].ident, } } - case 518: + case 519: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionLastBackupTSO, UintValue: yyS[yypt-0].item.(uint64), } } - case 519: + case 520: { // TODO: check overflow? parser.yyVAL.item = &ast.BRIEOption{ @@ -14016,21 +14032,21 @@ yynewstate: UintValue: yyS[yypt-3].item.(uint64) * 1048576, } } - case 520: + case 521: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionCSVHeader, UintValue: ast.BRIECSVHeaderIsColumns, } } - case 521: + case 522: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionCSVHeader, UintValue: yyS[yypt-0].item.(uint64), } } - case 522: + case 523: { value := uint64(0) if yyS[yypt-0].item.(bool) { @@ -14041,14 +14057,14 @@ yynewstate: UintValue: value, } } - case 523: + case 524: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionChecksum, UintValue: uint64(yyS[yypt-0].item.(ast.BRIEOptionLevel)), } } - case 524: + case 525: { value := uint64(0) if yyS[yypt-0].item.(bool) { @@ -14059,18 +14075,18 @@ yynewstate: UintValue: value, } } - case 525: + case 526: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionAnalyze, UintValue: uint64(yyS[yypt-0].item.(ast.BRIEOptionLevel)), } } - case 526: + case 527: { parser.yyVAL.item = getUint64FromNUM(yyS[yypt-0].item) } - case 527: + case 528: { v, rangeErrMsg := getInt64FromNUM(yyS[yypt-0].item) if len(rangeErrMsg) != 0 { @@ -14079,35 +14095,35 @@ yynewstate: } parser.yyVAL.item = v } - case 529: + case 530: { parser.yyVAL.item = yyS[yypt-0].item.(int64) != 0 } - case 530: + case 531: { parser.yyVAL.item = false } - case 531: + case 532: { parser.yyVAL.item = true } - case 532: + case 533: { parser.yyVAL.item = ast.BRIEOptionLevelOff } - case 533: + case 534: { parser.yyVAL.item = ast.BRIEOptionLevelOptional } - case 534: + case 535: { parser.yyVAL.item = ast.BRIEOptionLevelRequired } - case 535: + case 536: { parser.yyVAL.statement = &ast.PurgeImportStmt{TaskID: getUint64FromNUM(yyS[yypt-0].item)} } - case 536: + case 537: { parser.yyVAL.statement = &ast.CreateImportStmt{ IfNotExists: yyS[yypt-5].item.(bool), @@ -14117,21 +14133,21 @@ yynewstate: Options: yyS[yypt-0].item.([]*ast.BRIEOption), } } - case 537: + case 538: { parser.yyVAL.statement = &ast.StopImportStmt{ IfRunning: yyS[yypt-1].item.(bool), Name: yyS[yypt-0].ident, } } - case 538: + case 539: { parser.yyVAL.statement = &ast.ResumeImportStmt{ IfNotRunning: yyS[yypt-1].item.(bool), Name: yyS[yypt-0].ident, } } - case 539: + case 540: { s := &ast.AlterImportStmt{ Name: yyS[yypt-3].ident, @@ -14143,14 +14159,14 @@ yynewstate: } parser.yyVAL.statement = s } - case 540: + case 541: { parser.yyVAL.statement = &ast.DropImportStmt{ IfExists: yyS[yypt-1].item.(bool), Name: yyS[yypt-0].ident, } } - case 541: + case 542: { parser.yyVAL.statement = &ast.ShowImportStmt{ Name: yyS[yypt-2].ident, @@ -14158,73 +14174,73 @@ yynewstate: TableNames: yyS[yypt-0].item.([]*ast.TableName), } } - case 542: + case 543: { parser.yyVAL.item = false } - case 543: + case 544: { parser.yyVAL.item = true } - case 544: + case 545: { parser.yyVAL.item = false } - case 545: + case 546: { parser.yyVAL.item = true } - case 546: + case 547: { parser.yyVAL.item = false } - case 547: + case 548: { parser.yyVAL.item = true } - case 548: + case 549: { parser.yyVAL.item = ast.ErrorHandleError } - case 549: + case 550: { parser.yyVAL.item = ast.ErrorHandleReplace } - case 550: + case 551: { parser.yyVAL.item = ast.ErrorHandleSkipAll } - case 551: + case 552: { parser.yyVAL.item = ast.ErrorHandleSkipConstraint } - case 552: + case 553: { parser.yyVAL.item = ast.ErrorHandleSkipDuplicate } - case 553: + case 554: { parser.yyVAL.item = ast.ErrorHandleSkipStrict } - case 554: + case 555: { parser.yyVAL.item = nil } - case 555: + case 556: { parser.yyVAL.item = &ast.ImportTruncate{ IsErrorsOnly: false, TableNames: yyS[yypt-0].item.([]*ast.TableName), } } - case 556: + case 557: { parser.yyVAL.item = &ast.ImportTruncate{ IsErrorsOnly: true, TableNames: yyS[yypt-0].item.([]*ast.TableName), } } - case 557: + case 558: { v := yyS[yypt-2].ident v = strings.TrimPrefix(v, "@") @@ -14235,19 +14251,19 @@ yynewstate: Value: yyS[yypt-0].expr, } } - case 558: + case 559: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.LogicOr, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 559: + case 560: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.LogicXor, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 560: + case 561: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.LogicAnd, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 561: + case 562: { expr, ok := yyS[yypt-0].expr.(*ast.ExistsSubqueryExpr) if ok { @@ -14257,7 +14273,7 @@ yynewstate: parser.yyVAL.expr = &ast.UnaryOperationExpr{Op: opcode.Not, V: yyS[yypt-0].expr} } } - case 562: + case 563: { parser.yyVAL.expr = &ast.MatchAgainst{ ColumnNames: yyS[yypt-6].item.([]*ast.ColumnName), @@ -14265,87 +14281,87 @@ yynewstate: Modifier: ast.FulltextSearchModifier(yyS[yypt-1].item.(int)), } } - case 563: + case 564: { parser.yyVAL.expr = &ast.IsTruthExpr{Expr: yyS[yypt-2].expr, Not: !yyS[yypt-1].item.(bool), True: int64(1)} } - case 564: + case 565: { parser.yyVAL.expr = &ast.IsTruthExpr{Expr: yyS[yypt-2].expr, Not: !yyS[yypt-1].item.(bool), True: int64(0)} } - case 565: + case 566: { /* https://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#operator_is */ parser.yyVAL.expr = &ast.IsNullExpr{Expr: yyS[yypt-2].expr, Not: !yyS[yypt-1].item.(bool)} } - case 567: + case 568: { parser.yyVAL.expr = &ast.MaxValueExpr{} } - case 569: + case 570: { parser.yyVAL.item = ast.FulltextSearchModifierNaturalLanguageMode } - case 570: + case 571: { parser.yyVAL.item = ast.FulltextSearchModifierNaturalLanguageMode } - case 571: + case 572: { parser.yyVAL.item = ast.FulltextSearchModifierNaturalLanguageMode | ast.FulltextSearchModifierWithQueryExpansion } - case 572: + case 573: { parser.yyVAL.item = ast.FulltextSearchModifierBooleanMode } - case 573: + case 574: { parser.yyVAL.item = ast.FulltextSearchModifierWithQueryExpansion } - case 578: + case 579: { parser.yyVAL.item = []ast.ExprNode{yyS[yypt-0].expr} } - case 579: + case 580: { parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.ExprNode), yyS[yypt-0].expr) } - case 580: + case 581: { parser.yyVAL.item = []ast.ExprNode{yyS[yypt-0].expr} } - case 581: + case 582: { parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.ExprNode), yyS[yypt-0].expr) } - case 582: + case 583: { parser.yyVAL.item = []ast.ExprNode{} } - case 584: + case 585: { parser.yyVAL.item = []ast.ExprNode{} } - case 586: + case 587: { expr := ast.NewValueExpr(yyS[yypt-0].item, parser.charset, parser.collation) parser.yyVAL.item = []ast.ExprNode{expr} } - case 587: + case 588: { parser.yyVAL.expr = &ast.IsNullExpr{Expr: yyS[yypt-2].expr, Not: !yyS[yypt-1].item.(bool)} } - case 588: + case 589: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: yyS[yypt-1].item.(opcode.Op), L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 589: + case 590: { sq := yyS[yypt-0].expr.(*ast.SubqueryExpr) sq.MultiRows = true parser.yyVAL.expr = &ast.CompareSubqueryExpr{Op: yyS[yypt-2].item.(opcode.Op), L: yyS[yypt-3].expr, R: sq, All: yyS[yypt-1].item.(bool)} } - case 590: + case 591: { v := yyS[yypt-2].ident v = strings.TrimPrefix(v, "@") @@ -14357,25 +14373,21 @@ yynewstate: } parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: yyS[yypt-3].item.(opcode.Op), L: yyS[yypt-4].expr, R: variable} } - case 592: - { - parser.yyVAL.item = opcode.GE - } case 593: { - parser.yyVAL.item = opcode.GT + parser.yyVAL.item = opcode.GE } case 594: { - parser.yyVAL.item = opcode.LE + parser.yyVAL.item = opcode.GT } case 595: { - parser.yyVAL.item = opcode.LT + parser.yyVAL.item = opcode.LE } case 596: { - parser.yyVAL.item = opcode.NE + parser.yyVAL.item = opcode.LT } case 597: { @@ -14383,51 +14395,51 @@ yynewstate: } case 598: { - parser.yyVAL.item = opcode.EQ + parser.yyVAL.item = opcode.NE } case 599: { - parser.yyVAL.item = opcode.NullEQ + parser.yyVAL.item = opcode.EQ } case 600: { - parser.yyVAL.item = true + parser.yyVAL.item = opcode.NullEQ } case 601: { - parser.yyVAL.item = false + parser.yyVAL.item = true } case 602: { - parser.yyVAL.item = true + parser.yyVAL.item = false } case 603: { - parser.yyVAL.item = false + parser.yyVAL.item = true } case 604: { - parser.yyVAL.item = true + parser.yyVAL.item = false } case 605: { - parser.yyVAL.item = false + parser.yyVAL.item = true } case 606: { - parser.yyVAL.item = true + parser.yyVAL.item = false } case 607: { - parser.yyVAL.item = false + parser.yyVAL.item = true } case 608: { - parser.yyVAL.item = true + parser.yyVAL.item = false } case 609: { - parser.yyVAL.item = false + parser.yyVAL.item = true } case 610: { @@ -14439,19 +14451,23 @@ yynewstate: } case 612: { - parser.yyVAL.item = true + parser.yyVAL.item = false } case 613: { - parser.yyVAL.expr = &ast.PatternInExpr{Expr: yyS[yypt-4].expr, Not: !yyS[yypt-3].item.(bool), List: yyS[yypt-1].item.([]ast.ExprNode)} + parser.yyVAL.item = true } case 614: + { + parser.yyVAL.expr = &ast.PatternInExpr{Expr: yyS[yypt-4].expr, Not: !yyS[yypt-3].item.(bool), List: yyS[yypt-1].item.([]ast.ExprNode)} + } + case 615: { sq := yyS[yypt-0].expr.(*ast.SubqueryExpr) sq.MultiRows = true parser.yyVAL.expr = &ast.PatternInExpr{Expr: yyS[yypt-2].expr, Not: !yyS[yypt-1].item.(bool), Sel: sq} } - case 615: + case 616: { parser.yyVAL.expr = &ast.BetweenExpr{ Expr: yyS[yypt-4].expr, @@ -14460,7 +14476,7 @@ yynewstate: Not: !yyS[yypt-3].item.(bool), } } - case 616: + case 617: { escape := yyS[yypt-0].ident if len(escape) > 1 { @@ -14476,57 +14492,57 @@ yynewstate: Escape: escape[0], } } - case 617: + case 618: { parser.yyVAL.expr = &ast.PatternRegexpExpr{Expr: yyS[yypt-2].expr, Pattern: yyS[yypt-0].expr, Not: !yyS[yypt-1].item.(bool)} } - case 621: + case 622: { parser.yyVAL.ident = "\\" } - case 622: + case 623: { parser.yyVAL.ident = yyS[yypt-0].ident } - case 623: + case 624: { parser.yyVAL.item = &ast.SelectField{WildCard: &ast.WildCardField{}} } - case 624: + case 625: { wildCard := &ast.WildCardField{Table: model.NewCIStr(yyS[yypt-2].ident)} parser.yyVAL.item = &ast.SelectField{WildCard: wildCard} } - case 625: + case 626: { wildCard := &ast.WildCardField{Schema: model.NewCIStr(yyS[yypt-4].ident), Table: model.NewCIStr(yyS[yypt-2].ident)} parser.yyVAL.item = &ast.SelectField{WildCard: wildCard} } - case 626: + case 627: { expr := yyS[yypt-1].expr asName := yyS[yypt-0].ident parser.yyVAL.item = &ast.SelectField{Expr: expr, AsName: model.NewCIStr(asName)} } - case 627: + case 628: { parser.yyVAL.ident = "" } - case 630: + case 631: { parser.yyVAL.ident = yyS[yypt-0].ident } - case 632: + case 633: { parser.yyVAL.ident = yyS[yypt-0].ident } - case 633: + case 634: { field := yyS[yypt-0].item.(*ast.SelectField) field.Offset = parser.startOffset(&yyS[yypt]) parser.yyVAL.item = []*ast.SelectField{field} } - case 634: + case 635: { fl := yyS[yypt-2].item.([]*ast.SelectField) last := fl[len(fl)-1] @@ -14538,71 +14554,71 @@ yynewstate: newField.Offset = parser.startOffset(&yyS[yypt]) parser.yyVAL.item = append(fl, newField) } - case 635: + case 636: { parser.yyVAL.item = &ast.GroupByClause{Items: yyS[yypt-0].item.([]*ast.ByItem)} } - case 636: + case 637: { parser.yyVAL.item = nil } - case 637: + case 638: { parser.yyVAL.item = &ast.HavingClause{Expr: yyS[yypt-0].expr} } - case 638: + case 639: { parser.yyVAL.item = nil } - case 640: + case 641: { parser.yyVAL.item = &ast.AsOfClause{ TsExpr: yyS[yypt-0].expr.(ast.ExprNode), } } - case 641: + case 642: { parser.yyVAL.item = false } - case 642: + case 643: { parser.yyVAL.item = true } - case 643: + case 644: { parser.yyVAL.item = false } - case 644: + case 645: { parser.yyVAL.item = true } - case 645: + case 646: { parser.yyVAL.item = false } - case 646: + case 647: { parser.yyVAL.item = true } - case 647: + case 648: { parser.yyVAL.item = &ast.NullString{ String: "", Empty: false, } } - case 648: + case 649: { parser.yyVAL.item = &ast.NullString{ String: yyS[yypt-0].ident, Empty: len(yyS[yypt-0].ident) == 0, } } - case 649: + case 650: { parser.yyVAL.item = nil } - case 650: + case 651: { // Merge the options if yyS[yypt-1].item == nil { @@ -14626,19 +14642,19 @@ yynewstate: parser.yyVAL.item = opt1 } } - case 651: + case 652: { parser.yyVAL.item = &ast.IndexOption{ KeyBlockSize: yyS[yypt-0].item.(uint64), } } - case 652: + case 653: { parser.yyVAL.item = &ast.IndexOption{ Tp: yyS[yypt-0].item.(model.IndexType), } } - case 653: + case 654: { parser.yyVAL.item = &ast.IndexOption{ ParserName: model.NewCIStr(yyS[yypt-0].ident), @@ -14646,75 +14662,75 @@ yynewstate: yylex.AppendError(yylex.Errorf("The WITH PARASER clause is parsed but ignored by all storage engines.")) parser.lastErrorAsWarn() } - case 654: + case 655: { parser.yyVAL.item = &ast.IndexOption{ Comment: yyS[yypt-0].ident, } } - case 655: + case 656: { parser.yyVAL.item = &ast.IndexOption{ Visibility: yyS[yypt-0].item.(ast.IndexVisibility), } } - case 656: + case 657: { parser.yyVAL.item = &ast.IndexOption{ PrimaryKeyTp: yyS[yypt-0].item.(model.PrimaryKeyType), } } - case 657: + case 658: { parser.yyVAL.item = []interface{}{yyS[yypt-0].item, nil} } - case 658: + case 659: { parser.yyVAL.item = []interface{}{yyS[yypt-2].item, yyS[yypt-0].item} } - case 659: + case 660: { parser.yyVAL.item = []interface{}{&ast.NullString{String: yyS[yypt-2].ident, Empty: len(yyS[yypt-2].ident) == 0}, yyS[yypt-0].item} } - case 660: + case 661: { parser.yyVAL.item = nil } - case 662: + case 663: { parser.yyVAL.item = yyS[yypt-0].item } - case 663: + case 664: { parser.yyVAL.item = yyS[yypt-0].item } - case 664: + case 665: { parser.yyVAL.item = model.IndexTypeBtree } - case 665: + case 666: { parser.yyVAL.item = model.IndexTypeHash } - case 666: + case 667: { parser.yyVAL.item = model.IndexTypeRtree } - case 667: + case 668: { parser.yyVAL.item = ast.IndexVisibilityVisible } - case 668: + case 669: { parser.yyVAL.item = ast.IndexVisibilityInvisible } - case 1126: + case 1127: { parser.yyVAL.statement = &ast.CallStmt{ Procedure: yyS[yypt-0].expr.(*ast.FuncCallExpr), } } - case 1127: + case 1128: { parser.yyVAL.expr = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, @@ -14722,7 +14738,7 @@ yynewstate: Args: []ast.ExprNode{}, } } - case 1128: + case 1129: { parser.yyVAL.expr = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, @@ -14731,7 +14747,7 @@ yynewstate: Args: []ast.ExprNode{}, } } - case 1129: + case 1130: { parser.yyVAL.expr = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, @@ -14739,7 +14755,7 @@ yynewstate: Args: yyS[yypt-1].item.([]ast.ExprNode), } } - case 1130: + case 1131: { parser.yyVAL.expr = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, @@ -14748,7 +14764,7 @@ yynewstate: Args: yyS[yypt-1].item.([]ast.ExprNode), } } - case 1131: + case 1132: { x := yyS[yypt-1].item.(*ast.InsertStmt) x.Priority = yyS[yypt-6].item.(mysql.PriorityEnum) @@ -14765,17 +14781,13 @@ yynewstate: x.PartitionNames = yyS[yypt-2].item.([]model.CIStr) parser.yyVAL.statement = x } - case 1134: + case 1135: { parser.yyVAL.item = &ast.InsertStmt{ Columns: yyS[yypt-3].item.([]*ast.ColumnName), Lists: yyS[yypt-0].item.([][]ast.ExprNode), } } - case 1135: - { - parser.yyVAL.item = &ast.InsertStmt{Columns: yyS[yypt-2].item.([]*ast.ColumnName), Select: yyS[yypt-0].statement.(ast.ResultSetNode)} - } case 1136: { parser.yyVAL.item = &ast.InsertStmt{Columns: yyS[yypt-2].item.([]*ast.ColumnName), Select: yyS[yypt-0].statement.(ast.ResultSetNode)} @@ -14785,6 +14797,10 @@ yynewstate: parser.yyVAL.item = &ast.InsertStmt{Columns: yyS[yypt-2].item.([]*ast.ColumnName), Select: yyS[yypt-0].statement.(ast.ResultSetNode)} } case 1138: + { + parser.yyVAL.item = &ast.InsertStmt{Columns: yyS[yypt-2].item.([]*ast.ColumnName), Select: yyS[yypt-0].statement.(ast.ResultSetNode)} + } + case 1139: { var sel ast.ResultSetNode switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -14797,13 +14813,9 @@ yynewstate: } parser.yyVAL.item = &ast.InsertStmt{Columns: yyS[yypt-2].item.([]*ast.ColumnName), Select: sel} } - case 1139: - { - parser.yyVAL.item = &ast.InsertStmt{Lists: yyS[yypt-0].item.([][]ast.ExprNode)} - } case 1140: { - parser.yyVAL.item = &ast.InsertStmt{Select: yyS[yypt-0].statement.(ast.ResultSetNode)} + parser.yyVAL.item = &ast.InsertStmt{Lists: yyS[yypt-0].item.([][]ast.ExprNode)} } case 1141: { @@ -14814,6 +14826,10 @@ yynewstate: parser.yyVAL.item = &ast.InsertStmt{Select: yyS[yypt-0].statement.(ast.ResultSetNode)} } case 1143: + { + parser.yyVAL.item = &ast.InsertStmt{Select: yyS[yypt-0].statement.(ast.ResultSetNode)} + } + case 1144: { var sel ast.ResultSetNode switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -14826,66 +14842,66 @@ yynewstate: } parser.yyVAL.item = &ast.InsertStmt{Select: sel} } - case 1144: + case 1145: { parser.yyVAL.item = &ast.InsertStmt{Setlist: yyS[yypt-0].item.([]*ast.Assignment)} } - case 1147: + case 1148: { parser.yyVAL.item = [][]ast.ExprNode{yyS[yypt-0].item.([]ast.ExprNode)} } - case 1148: + case 1149: { parser.yyVAL.item = append(yyS[yypt-2].item.([][]ast.ExprNode), yyS[yypt-0].item.([]ast.ExprNode)) } - case 1149: + case 1150: { parser.yyVAL.item = yyS[yypt-1].item } - case 1150: + case 1151: { parser.yyVAL.item = []ast.ExprNode{} } - case 1152: + case 1153: { parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.ExprNode), yyS[yypt-0].expr) } - case 1153: + case 1154: { parser.yyVAL.item = []ast.ExprNode{yyS[yypt-0].expr} } - case 1155: + case 1156: { parser.yyVAL.expr = &ast.DefaultExpr{} } - case 1156: + case 1157: { parser.yyVAL.item = &ast.Assignment{ Column: yyS[yypt-2].item.(*ast.ColumnName), Expr: yyS[yypt-0].expr, } } - case 1157: + case 1158: { parser.yyVAL.item = []*ast.Assignment{} } - case 1158: + case 1159: { parser.yyVAL.item = []*ast.Assignment{yyS[yypt-0].item.(*ast.Assignment)} } - case 1159: + case 1160: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*ast.Assignment), yyS[yypt-0].item.(*ast.Assignment)) } - case 1160: + case 1161: { parser.yyVAL.item = nil } - case 1161: + case 1162: { parser.yyVAL.item = yyS[yypt-0].item } - case 1162: + case 1163: { x := yyS[yypt-0].item.(*ast.InsertStmt) x.IsReplace = true @@ -14895,31 +14911,31 @@ yynewstate: x.PartitionNames = yyS[yypt-1].item.([]model.CIStr) parser.yyVAL.statement = x } - case 1163: + case 1164: { parser.yyVAL.expr = ast.NewValueExpr(false, parser.charset, parser.collation) } - case 1164: + case 1165: { parser.yyVAL.expr = ast.NewValueExpr(nil, parser.charset, parser.collation) } - case 1165: + case 1166: { parser.yyVAL.expr = ast.NewValueExpr(true, parser.charset, parser.collation) } - case 1166: + case 1167: { parser.yyVAL.expr = ast.NewValueExpr(yyS[yypt-0].item, parser.charset, parser.collation) } - case 1167: + case 1168: { parser.yyVAL.expr = ast.NewValueExpr(yyS[yypt-0].item, parser.charset, parser.collation) } - case 1168: + case 1169: { parser.yyVAL.expr = ast.NewValueExpr(yyS[yypt-0].item, parser.charset, parser.collation) } - case 1170: + case 1171: { // See https://dev.mysql.com/doc/refman/5.7/en/charset-literal.html co, err := charset.GetDefaultCollationLegacy(yyS[yypt-1].ident) @@ -14936,15 +14952,15 @@ yynewstate: } parser.yyVAL.expr = expr } - case 1171: + case 1172: { parser.yyVAL.expr = ast.NewValueExpr(yyS[yypt-0].item, parser.charset, parser.collation) } - case 1172: + case 1173: { parser.yyVAL.expr = ast.NewValueExpr(yyS[yypt-0].item, parser.charset, parser.collation) } - case 1173: + case 1174: { co, err := charset.GetDefaultCollationLegacy(yyS[yypt-1].ident) if err != nil { @@ -14960,7 +14976,7 @@ yynewstate: } parser.yyVAL.expr = expr } - case 1174: + case 1175: { co, err := charset.GetDefaultCollationLegacy(yyS[yypt-1].ident) if err != nil { @@ -14976,12 +14992,12 @@ yynewstate: } parser.yyVAL.expr = expr } - case 1175: + case 1176: { expr := ast.NewValueExpr(yyS[yypt-0].ident, parser.charset, parser.collation) parser.yyVAL.expr = expr } - case 1176: + case 1177: { valExpr := yyS[yypt-1].expr.(ast.ValueExpr) strLit := valExpr.GetString() @@ -14994,31 +15010,31 @@ yynewstate: } parser.yyVAL.expr = expr } - case 1177: + case 1178: { parser.yyVAL.item = []*ast.AlterOrderItem{yyS[yypt-0].item.(*ast.AlterOrderItem)} } - case 1178: + case 1179: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*ast.AlterOrderItem), yyS[yypt-0].item.(*ast.AlterOrderItem)) } - case 1179: + case 1180: { parser.yyVAL.item = &ast.AlterOrderItem{Column: yyS[yypt-1].item.(*ast.ColumnName), Desc: yyS[yypt-0].item.(bool)} } - case 1180: + case 1181: { parser.yyVAL.item = &ast.OrderByClause{Items: yyS[yypt-0].item.([]*ast.ByItem)} } - case 1181: + case 1182: { parser.yyVAL.item = []*ast.ByItem{yyS[yypt-0].item.(*ast.ByItem)} } - case 1182: + case 1183: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*ast.ByItem), yyS[yypt-0].item.(*ast.ByItem)) } - case 1183: + case 1184: { expr := yyS[yypt-0].expr valueExpr, ok := expr.(ast.ValueExpr) @@ -15030,7 +15046,7 @@ yynewstate: } parser.yyVAL.item = &ast.ByItem{Expr: expr, NullOrder: true} } - case 1184: + case 1185: { expr := yyS[yypt-1].expr valueExpr, ok := expr.(ast.ValueExpr) @@ -15042,55 +15058,55 @@ yynewstate: } parser.yyVAL.item = &ast.ByItem{Expr: expr, Desc: yyS[yypt-0].item.(bool)} } - case 1185: + case 1186: { parser.yyVAL.item = false } - case 1186: + case 1187: { parser.yyVAL.item = true } - case 1187: + case 1188: { parser.yyVAL.item = false // ASC by default } - case 1188: + case 1189: { parser.yyVAL.item = false } - case 1189: + case 1190: { parser.yyVAL.item = true } - case 1190: + case 1191: { parser.yyVAL.item = nil } - case 1192: + case 1193: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Or, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1193: + case 1194: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.And, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1194: + case 1195: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.LeftShift, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1195: + case 1196: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.RightShift, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1196: + case 1197: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Plus, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1197: + case 1198: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Minus, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1198: + case 1199: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr("DATE_ADD"), @@ -15101,7 +15117,7 @@ yynewstate: }, } } - case 1199: + case 1200: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr("DATE_SUB"), @@ -15112,44 +15128,44 @@ yynewstate: }, } } - case 1200: + case 1201: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Mul, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1201: + case 1202: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Div, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1202: + case 1203: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Mod, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1203: + case 1204: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.IntDiv, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1204: + case 1205: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Mod, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1205: + case 1206: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Xor, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1207: + case 1208: { parser.yyVAL.expr = &ast.ColumnNameExpr{Name: &ast.ColumnName{ Name: model.NewCIStr(yyS[yypt-0].ident), }} } - case 1208: + case 1209: { parser.yyVAL.expr = &ast.ColumnNameExpr{Name: &ast.ColumnName{ Table: model.NewCIStr(yyS[yypt-2].ident), Name: model.NewCIStr(yyS[yypt-0].ident), }} } - case 1209: + case 1210: { parser.yyVAL.expr = &ast.ColumnNameExpr{Name: &ast.ColumnName{ Schema: model.NewCIStr(yyS[yypt-4].ident), @@ -15157,39 +15173,39 @@ yynewstate: Name: model.NewCIStr(yyS[yypt-0].ident), }} } - case 1214: + case 1215: { parser.yyVAL.expr = &ast.SetCollationExpr{Expr: yyS[yypt-2].expr, Collate: yyS[yypt-0].ident} } - case 1217: + case 1218: { parser.yyVAL.expr = ast.NewParamMarkerExpr(yyS[yypt].offset) } - case 1220: + case 1221: { parser.yyVAL.expr = &ast.UnaryOperationExpr{Op: opcode.Not2, V: yyS[yypt-0].expr} } - case 1221: + case 1222: { parser.yyVAL.expr = &ast.UnaryOperationExpr{Op: opcode.BitNeg, V: yyS[yypt-0].expr} } - case 1222: + case 1223: { parser.yyVAL.expr = &ast.UnaryOperationExpr{Op: opcode.Minus, V: yyS[yypt-0].expr} } - case 1223: + case 1224: { parser.yyVAL.expr = &ast.UnaryOperationExpr{Op: opcode.Plus, V: yyS[yypt-0].expr} } - case 1224: + case 1225: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.Concat), Args: []ast.ExprNode{yyS[yypt-2].expr, yyS[yypt-0].expr}} } - case 1225: + case 1226: { parser.yyVAL.expr = &ast.UnaryOperationExpr{Op: opcode.Not2, V: yyS[yypt-0].expr} } - case 1227: + case 1228: { startOffset := parser.startOffset(&yyS[yypt-1]) endOffset := parser.endOffset(&yyS[yypt]) @@ -15197,23 +15213,23 @@ yynewstate: expr.SetText(parser.src[startOffset:endOffset]) parser.yyVAL.expr = &ast.ParenthesesExpr{Expr: expr} } - case 1228: + case 1229: { values := append(yyS[yypt-3].item.([]ast.ExprNode), yyS[yypt-1].expr) parser.yyVAL.expr = &ast.RowExpr{Values: values} } - case 1229: + case 1230: { values := append(yyS[yypt-3].item.([]ast.ExprNode), yyS[yypt-1].expr) parser.yyVAL.expr = &ast.RowExpr{Values: values} } - case 1230: + case 1231: { sq := yyS[yypt-0].expr.(*ast.SubqueryExpr) sq.Exists = true parser.yyVAL.expr = &ast.ExistsSubqueryExpr{Sel: sq} } - case 1231: + case 1232: { /* * ODBC escape syntax. @@ -15237,7 +15253,7 @@ yynewstate: parser.yyVAL.expr = yyS[yypt-1].expr } } - case 1232: + case 1233: { // See https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#operator_binary x := types.NewFieldType(mysql.TypeString) @@ -15250,7 +15266,7 @@ yynewstate: FunctionType: ast.CastBinaryOperator, } } - case 1233: + case 1234: { /* See https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#function_cast */ tp := yyS[yypt-1].item.(*types.FieldType) @@ -15270,7 +15286,7 @@ yynewstate: ExplicitCharSet: explicitCharset, } } - case 1234: + case 1235: { x := &ast.CaseExpr{WhenClauses: yyS[yypt-2].item.([]*ast.WhenClause)} if yyS[yypt-3].expr != nil { @@ -15281,7 +15297,7 @@ yynewstate: } parser.yyVAL.expr = x } - case 1235: + case 1236: { // See https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#function_convert tp := yyS[yypt-1].item.(*types.FieldType) @@ -15301,7 +15317,7 @@ yynewstate: ExplicitCharSet: explicitCharset, } } - case 1236: + case 1237: { // See https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#function_convert charset1 := ast.NewValueExpr(yyS[yypt-1].ident, "", "") @@ -15310,62 +15326,62 @@ yynewstate: Args: []ast.ExprNode{yyS[yypt-3].expr, charset1}, } } - case 1237: + case 1238: { parser.yyVAL.expr = &ast.DefaultExpr{Name: yyS[yypt-1].expr.(*ast.ColumnNameExpr).Name} } - case 1238: + case 1239: { parser.yyVAL.expr = &ast.ValuesExpr{Column: yyS[yypt-1].expr.(*ast.ColumnNameExpr)} } - case 1239: + case 1240: { expr := ast.NewValueExpr(yyS[yypt-0].ident, parser.charset, parser.collation) parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.JSONExtract), Args: []ast.ExprNode{yyS[yypt-2].expr, expr}} } - case 1240: + case 1241: { expr := ast.NewValueExpr(yyS[yypt-0].ident, parser.charset, parser.collation) extract := &ast.FuncCallExpr{FnName: model.NewCIStr(ast.JSONExtract), Args: []ast.ExprNode{yyS[yypt-2].expr, expr}} parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.JSONUnquote), Args: []ast.ExprNode{extract}} } - case 1243: + case 1244: { parser.yyVAL.item = false } - case 1244: + case 1245: { parser.yyVAL.item = true } - case 1245: + case 1246: { parser.yyVAL.item = false } - case 1247: + case 1248: { parser.yyVAL.item = true } - case 1250: + case 1251: { parser.yyVAL.item = true } - case 1292: + case 1293: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} } - case 1293: + case 1294: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} } - case 1294: + case 1295: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-1].ident)} } - case 1295: + case 1296: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-2].ident)} } - case 1296: + case 1297: { args := []ast.ExprNode{} if yyS[yypt-0].item != nil { @@ -15373,7 +15389,7 @@ yynewstate: } parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-1].ident), Args: args} } - case 1297: + case 1298: { nilVal := ast.NewValueExpr(nil, parser.charset, parser.collation) args := yyS[yypt-1].item.([]ast.ExprNode) @@ -15382,7 +15398,7 @@ yynewstate: Args: append(args, nilVal), } } - case 1298: + case 1299: { charset1 := ast.NewValueExpr(yyS[yypt-1].ident, "", "") args := yyS[yypt-3].item.([]ast.ExprNode) @@ -15391,42 +15407,42 @@ yynewstate: Args: append(args, charset1), } } - case 1299: + case 1300: { expr := ast.NewValueExpr(yyS[yypt-0].ident, "", "") parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.DateLiteral), Args: []ast.ExprNode{expr}} } - case 1300: + case 1301: { expr := ast.NewValueExpr(yyS[yypt-0].ident, "", "") parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.TimeLiteral), Args: []ast.ExprNode{expr}} } - case 1301: + case 1302: { expr := ast.NewValueExpr(yyS[yypt-0].ident, "", "") parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.TimestampLiteral), Args: []ast.ExprNode{expr}} } - case 1302: + case 1303: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.InsertFunc), Args: yyS[yypt-1].item.([]ast.ExprNode)} } - case 1303: + case 1304: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Mod, L: yyS[yypt-3].expr, R: yyS[yypt-1].expr} } - case 1304: + case 1305: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.PasswordFunc), Args: yyS[yypt-1].item.([]ast.ExprNode)} } - case 1305: + case 1306: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} } - case 1306: + case 1307: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} } - case 1307: + case 1308: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-5].ident), @@ -15437,7 +15453,7 @@ yynewstate: }, } } - case 1308: + case 1309: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-7].ident), @@ -15448,7 +15464,7 @@ yynewstate: }, } } - case 1309: + case 1310: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-7].ident), @@ -15459,7 +15475,7 @@ yynewstate: }, } } - case 1310: + case 1311: { timeUnit := &ast.TimeUnitExpr{Unit: yyS[yypt-3].item.(ast.TimeUnitType)} parser.yyVAL.expr = &ast.FuncCallExpr{ @@ -15467,7 +15483,7 @@ yynewstate: Args: []ast.ExprNode{timeUnit, yyS[yypt-1].expr}, } } - case 1311: + case 1312: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-5].ident), @@ -15477,67 +15493,67 @@ yynewstate: }, } } - case 1312: + case 1313: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{yyS[yypt-3].expr, yyS[yypt-1].expr}} } - case 1313: + case 1314: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{yyS[yypt-3].expr, yyS[yypt-1].expr}, } } - case 1314: + case 1315: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{yyS[yypt-3].expr, yyS[yypt-1].expr}, } } - case 1315: + case 1316: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-3].expr, yyS[yypt-1].expr}, } } - case 1316: + case 1317: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-3].expr, yyS[yypt-1].expr}, } } - case 1317: + case 1318: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{&ast.TimeUnitExpr{Unit: yyS[yypt-5].item.(ast.TimeUnitType)}, yyS[yypt-3].expr, yyS[yypt-1].expr}, } } - case 1318: + case 1319: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{&ast.TimeUnitExpr{Unit: yyS[yypt-5].item.(ast.TimeUnitType)}, yyS[yypt-3].expr, yyS[yypt-1].expr}, } } - case 1319: + case 1320: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-3].ident), Args: []ast.ExprNode{yyS[yypt-1].expr}, } } - case 1320: + case 1321: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{yyS[yypt-1].expr, yyS[yypt-3].expr}, } } - case 1321: + case 1322: { spaceVal := ast.NewValueExpr(" ", parser.charset, parser.collation) direction := &ast.TrimDirectionExpr{Direction: yyS[yypt-3].item.(ast.TrimDirectionType)} @@ -15546,7 +15562,7 @@ yynewstate: Args: []ast.ExprNode{yyS[yypt-1].expr, spaceVal, direction}, } } - case 1322: + case 1323: { direction := &ast.TrimDirectionExpr{Direction: yyS[yypt-4].item.(ast.TrimDirectionType)} parser.yyVAL.expr = &ast.FuncCallExpr{ @@ -15554,63 +15570,63 @@ yynewstate: Args: []ast.ExprNode{yyS[yypt-1].expr, yyS[yypt-3].expr, direction}, } } - case 1323: + case 1324: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-3].ident), Args: []ast.ExprNode{yyS[yypt-1].expr}, } } - case 1324: + case 1325: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-6].ident), Args: []ast.ExprNode{yyS[yypt-4].expr, ast.NewValueExpr("CHAR", parser.charset, parser.collation), ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)}, } } - case 1325: + case 1326: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-6].ident), Args: []ast.ExprNode{yyS[yypt-4].expr, ast.NewValueExpr("BINARY", parser.charset, parser.collation), ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)}, } } - case 1327: + case 1328: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-3].expr, yyS[yypt-1].expr}, } } - case 1328: + case 1329: { parser.yyVAL.item = ast.GetFormatSelectorDate } - case 1329: + case 1330: { parser.yyVAL.item = ast.GetFormatSelectorDatetime } - case 1330: + case 1331: { parser.yyVAL.item = ast.GetFormatSelectorTime } - case 1331: + case 1332: { parser.yyVAL.item = ast.GetFormatSelectorDatetime } - case 1336: + case 1337: { parser.yyVAL.item = ast.TrimBoth } - case 1337: + case 1338: { parser.yyVAL.item = ast.TrimLeading } - case 1338: + case 1339: { parser.yyVAL.item = ast.TrimTrailing } - case 1339: + case 1340: { objNameExpr := &ast.TableNameExpr{ Name: yyS[yypt-1].item.(*ast.TableName), @@ -15620,7 +15636,7 @@ yynewstate: Args: []ast.ExprNode{objNameExpr}, } } - case 1340: + case 1341: { objNameExpr := &ast.TableNameExpr{ Name: yyS[yypt-3].item.(*ast.TableName), @@ -15631,7 +15647,7 @@ yynewstate: Args: []ast.ExprNode{objNameExpr, valueExpr}, } } - case 1342: + case 1343: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool), Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15639,15 +15655,15 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } } - case 1343: + case 1344: { parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-3].ident, Args: yyS[yypt-1].item.([]ast.ExprNode), Distinct: false} } - case 1344: + case 1345: { parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-3].ident, Args: yyS[yypt-1].item.([]ast.ExprNode)} } - case 1345: + case 1346: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15655,7 +15671,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1346: + case 1347: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15663,7 +15679,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1347: + case 1348: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15671,7 +15687,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1348: + case 1349: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15679,7 +15695,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1349: + case 1350: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15687,7 +15703,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1350: + case 1351: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15695,11 +15711,11 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1351: + case 1352: { parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-4].ident, Args: yyS[yypt-1].item.([]ast.ExprNode), Distinct: true} } - case 1352: + case 1353: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15707,7 +15723,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1353: + case 1354: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15715,7 +15731,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1354: + case 1355: { args := []ast.ExprNode{ast.NewValueExpr(1, parser.charset, parser.collation)} if yyS[yypt-0].item != nil { @@ -15724,7 +15740,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-4].ident, Args: args} } } - case 1355: + case 1356: { args := yyS[yypt-4].item.([]ast.ExprNode) args = append(args, yyS[yypt-2].item.(ast.ExprNode)) @@ -15738,7 +15754,7 @@ yynewstate: parser.yyVAL.expr = agg } } - case 1356: + case 1357: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool), Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15746,7 +15762,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } } - case 1357: + case 1358: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool), Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15754,7 +15770,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } } - case 1358: + case 1359: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool), Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15762,7 +15778,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } } - case 1359: + case 1360: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: ast.AggFuncStddevPop, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool), Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15770,7 +15786,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: ast.AggFuncStddevPop, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } } - case 1360: + case 1361: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool), Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15778,7 +15794,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } } - case 1361: + case 1362: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: ast.AggFuncVarPop, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool), Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15786,11 +15802,11 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: ast.AggFuncVarPop, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } } - case 1362: + case 1363: { parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } - case 1363: + case 1364: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15798,7 +15814,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1364: + case 1365: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15806,7 +15822,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1365: + case 1366: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-6].ident, Args: []ast.ExprNode{yyS[yypt-4].expr, yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15814,7 +15830,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-6].ident, Args: []ast.ExprNode{yyS[yypt-4].expr, yyS[yypt-2].expr}} } } - case 1366: + case 1367: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-7].ident, Args: []ast.ExprNode{yyS[yypt-4].expr, yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15822,7 +15838,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-7].ident, Args: []ast.ExprNode{yyS[yypt-4].expr, yyS[yypt-2].expr}} } } - case 1367: + case 1368: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-7].ident, Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15830,7 +15846,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-7].ident, Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-2].expr}} } } - case 1368: + case 1369: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-8].ident, Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15838,22 +15854,22 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-8].ident, Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-2].expr}} } } - case 1369: + case 1370: { parser.yyVAL.item = ast.NewValueExpr(",", "", "") } - case 1370: + case 1371: { parser.yyVAL.item = ast.NewValueExpr(yyS[yypt-0].ident, "", "") } - case 1371: + case 1372: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode), } } - case 1372: + case 1373: { var tp ast.FuncCallExprType if isInTokenMap(yyS[yypt-3].ident) { @@ -15868,159 +15884,159 @@ yynewstate: Args: yyS[yypt-1].item.([]ast.ExprNode), } } - case 1373: + case 1374: { parser.yyVAL.item = nil } - case 1374: + case 1375: { parser.yyVAL.item = nil } - case 1375: + case 1376: { expr := ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation) parser.yyVAL.item = expr } - case 1377: + case 1378: { parser.yyVAL.item = ast.TimeUnitSecondMicrosecond } - case 1378: + case 1379: { parser.yyVAL.item = ast.TimeUnitMinuteMicrosecond } - case 1379: + case 1380: { parser.yyVAL.item = ast.TimeUnitMinuteSecond } - case 1380: + case 1381: { parser.yyVAL.item = ast.TimeUnitHourMicrosecond } - case 1381: + case 1382: { parser.yyVAL.item = ast.TimeUnitHourSecond } - case 1382: + case 1383: { parser.yyVAL.item = ast.TimeUnitHourMinute } - case 1383: + case 1384: { parser.yyVAL.item = ast.TimeUnitDayMicrosecond } - case 1384: + case 1385: { parser.yyVAL.item = ast.TimeUnitDaySecond } - case 1385: + case 1386: { parser.yyVAL.item = ast.TimeUnitDayMinute } - case 1386: + case 1387: { parser.yyVAL.item = ast.TimeUnitDayHour } - case 1387: + case 1388: { parser.yyVAL.item = ast.TimeUnitYearMonth } - case 1388: + case 1389: { parser.yyVAL.item = ast.TimeUnitMicrosecond } - case 1389: + case 1390: { parser.yyVAL.item = ast.TimeUnitSecond } - case 1390: + case 1391: { parser.yyVAL.item = ast.TimeUnitMinute } - case 1391: + case 1392: { parser.yyVAL.item = ast.TimeUnitHour } - case 1392: + case 1393: { parser.yyVAL.item = ast.TimeUnitDay } - case 1393: + case 1394: { parser.yyVAL.item = ast.TimeUnitWeek } - case 1394: + case 1395: { parser.yyVAL.item = ast.TimeUnitMonth } - case 1395: + case 1396: { parser.yyVAL.item = ast.TimeUnitQuarter } - case 1396: + case 1397: { parser.yyVAL.item = ast.TimeUnitYear } - case 1397: + case 1398: { parser.yyVAL.item = ast.TimeUnitSecond } - case 1398: + case 1399: { parser.yyVAL.item = ast.TimeUnitMinute } - case 1399: + case 1400: { parser.yyVAL.item = ast.TimeUnitHour } - case 1400: + case 1401: { parser.yyVAL.item = ast.TimeUnitDay } - case 1401: + case 1402: { parser.yyVAL.item = ast.TimeUnitWeek } - case 1402: + case 1403: { parser.yyVAL.item = ast.TimeUnitMonth } - case 1403: + case 1404: { parser.yyVAL.item = ast.TimeUnitQuarter } - case 1404: + case 1405: { parser.yyVAL.item = ast.TimeUnitYear } - case 1405: + case 1406: { parser.yyVAL.expr = nil } - case 1407: + case 1408: { parser.yyVAL.item = []*ast.WhenClause{yyS[yypt-0].item.(*ast.WhenClause)} } - case 1408: + case 1409: { parser.yyVAL.item = append(yyS[yypt-1].item.([]*ast.WhenClause), yyS[yypt-0].item.(*ast.WhenClause)) } - case 1409: + case 1410: { parser.yyVAL.item = &ast.WhenClause{ Expr: yyS[yypt-2].expr, Result: yyS[yypt-0].expr, } } - case 1410: + case 1411: { parser.yyVAL.item = nil } - case 1411: + case 1412: { parser.yyVAL.item = yyS[yypt-0].expr } - case 1412: + case 1413: { x := types.NewFieldType(mysql.TypeVarString) x.Flen = yyS[yypt-0].item.(int) // TODO: Flen should be the flen of expression @@ -16032,7 +16048,7 @@ yynewstate: x.Flag |= mysql.BinaryFlag parser.yyVAL.item = x } - case 1413: + case 1414: { x := types.NewFieldType(mysql.TypeVarString) x.Flen = yyS[yypt-1].item.(int) // TODO: Flen should be the flen of expression @@ -16055,7 +16071,7 @@ yynewstate: } parser.yyVAL.item = x } - case 1414: + case 1415: { x := types.NewFieldType(mysql.TypeDate) x.Charset = charset.CharsetBin @@ -16063,7 +16079,7 @@ yynewstate: x.Flag |= mysql.BinaryFlag parser.yyVAL.item = x } - case 1415: + case 1416: { x := types.NewFieldType(mysql.TypeYear) x.Charset = charset.CharsetBin @@ -16071,7 +16087,7 @@ yynewstate: x.Flag |= mysql.BinaryFlag parser.yyVAL.item = x } - case 1416: + case 1417: { x := types.NewFieldType(mysql.TypeDatetime) x.Flen, _ = mysql.GetDefaultFieldLengthAndDecimalForCast(mysql.TypeDatetime) @@ -16084,7 +16100,7 @@ yynewstate: x.Flag |= mysql.BinaryFlag parser.yyVAL.item = x } - case 1417: + case 1418: { fopt := yyS[yypt-0].item.(*ast.FloatOpt) x := types.NewFieldType(mysql.TypeNewDecimal) @@ -16095,7 +16111,7 @@ yynewstate: x.Flag |= mysql.BinaryFlag parser.yyVAL.item = x } - case 1418: + case 1419: { x := types.NewFieldType(mysql.TypeDuration) x.Flen, _ = mysql.GetDefaultFieldLengthAndDecimalForCast(mysql.TypeDuration) @@ -16108,7 +16124,7 @@ yynewstate: x.Flag |= mysql.BinaryFlag parser.yyVAL.item = x } - case 1419: + case 1420: { x := types.NewFieldType(mysql.TypeLonglong) x.Charset = charset.CharsetBin @@ -16116,7 +16132,7 @@ yynewstate: x.Flag |= mysql.BinaryFlag parser.yyVAL.item = x } - case 1420: + case 1421: { x := types.NewFieldType(mysql.TypeLonglong) x.Flag |= mysql.UnsignedFlag | mysql.BinaryFlag @@ -16124,7 +16140,7 @@ yynewstate: x.Collate = charset.CollationBin parser.yyVAL.item = x } - case 1421: + case 1422: { x := types.NewFieldType(mysql.TypeJSON) x.Flag |= mysql.BinaryFlag | (mysql.ParseToJSONFlag) @@ -16132,7 +16148,7 @@ yynewstate: x.Collate = mysql.DefaultCollationName parser.yyVAL.item = x } - case 1422: + case 1423: { x := types.NewFieldType(mysql.TypeDouble) x.Flen, x.Decimal = mysql.GetDefaultFieldLengthAndDecimalForCast(mysql.TypeDouble) @@ -16141,7 +16157,7 @@ yynewstate: x.Collate = charset.CollationBin parser.yyVAL.item = x } - case 1423: + case 1424: { x := types.NewFieldType(mysql.TypeFloat) fopt := yyS[yypt-0].item.(*ast.FloatOpt) @@ -16156,7 +16172,7 @@ yynewstate: x.Collate = charset.CollationBin parser.yyVAL.item = x } - case 1424: + case 1425: { var x *types.FieldType if parser.lexer.GetSQLMode().HasRealAsFloatMode() { @@ -16170,65 +16186,65 @@ yynewstate: x.Collate = charset.CollationBin parser.yyVAL.item = x } - case 1425: + case 1426: { parser.yyVAL.item = mysql.LowPriority } - case 1426: + case 1427: { parser.yyVAL.item = mysql.HighPriority } - case 1427: + case 1428: { parser.yyVAL.item = mysql.DelayedPriority } - case 1428: + case 1429: { parser.yyVAL.item = mysql.NoPriority } - case 1430: + case 1431: { parser.yyVAL.item = &ast.TableName{Name: model.NewCIStr(yyS[yypt-0].ident)} } - case 1431: + case 1432: { parser.yyVAL.item = &ast.TableName{Schema: model.NewCIStr(yyS[yypt-2].ident), Name: model.NewCIStr(yyS[yypt-0].ident)} } - case 1432: + case 1433: { tbl := []*ast.TableName{yyS[yypt-0].item.(*ast.TableName)} parser.yyVAL.item = tbl } - case 1433: + case 1434: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*ast.TableName), yyS[yypt-0].item.(*ast.TableName)) } - case 1434: + case 1435: { parser.yyVAL.item = &ast.TableName{Name: model.NewCIStr(yyS[yypt-1].ident)} } - case 1435: + case 1436: { parser.yyVAL.item = &ast.TableName{Schema: model.NewCIStr(yyS[yypt-3].ident), Name: model.NewCIStr(yyS[yypt-1].ident)} } - case 1436: + case 1437: { tbl := []*ast.TableName{yyS[yypt-0].item.(*ast.TableName)} parser.yyVAL.item = tbl } - case 1437: + case 1438: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*ast.TableName), yyS[yypt-0].item.(*ast.TableName)) } - case 1440: + case 1441: { parser.yyVAL.item = false } - case 1441: + case 1442: { parser.yyVAL.item = true } - case 1442: + case 1443: { var sqlText string var sqlVar *ast.VariableExpr @@ -16244,86 +16260,86 @@ yynewstate: SQLVar: sqlVar, } } - case 1443: + case 1444: { parser.yyVAL.item = yyS[yypt-0].ident } - case 1444: + case 1445: { parser.yyVAL.item = yyS[yypt-0].expr } - case 1445: + case 1446: { parser.yyVAL.statement = &ast.ExecuteStmt{Name: yyS[yypt-0].ident} } - case 1446: + case 1447: { parser.yyVAL.statement = &ast.ExecuteStmt{ Name: yyS[yypt-2].ident, UsingVars: yyS[yypt-0].item.([]ast.ExprNode), } } - case 1447: + case 1448: { parser.yyVAL.item = []ast.ExprNode{yyS[yypt-0].expr} } - case 1448: + case 1449: { parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.ExprNode), yyS[yypt-0].expr) } - case 1449: + case 1450: { parser.yyVAL.statement = &ast.DeallocateStmt{Name: yyS[yypt-0].ident} } - case 1452: + case 1453: { parser.yyVAL.statement = &ast.RollbackStmt{} } - case 1453: + case 1454: { parser.yyVAL.statement = &ast.RollbackStmt{CompletionType: yyS[yypt-0].item.(ast.CompletionType)} } - case 1454: + case 1455: { parser.yyVAL.item = ast.CompletionTypeChain } - case 1455: + case 1456: { parser.yyVAL.item = ast.CompletionTypeRelease } - case 1456: + case 1457: { parser.yyVAL.item = ast.CompletionTypeDefault } - case 1457: + case 1458: { parser.yyVAL.item = ast.CompletionTypeChain } - case 1458: + case 1459: { parser.yyVAL.item = ast.CompletionTypeDefault } - case 1459: + case 1460: { parser.yyVAL.item = ast.CompletionTypeRelease } - case 1460: + case 1461: { parser.yyVAL.item = ast.CompletionTypeDefault } - case 1461: + case 1462: { parser.yyVAL.statement = &ast.ShutdownStmt{} } - case 1462: + case 1463: { parser.yyVAL.statement = &ast.RestartStmt{} } - case 1463: + case 1464: { parser.yyVAL.statement = &ast.HelpStmt{Topic: yyS[yypt-0].ident} } - case 1464: + case 1465: { st := &ast.SelectStmt{ SelectStmtOpts: yyS[yypt-1].item.(*ast.SelectStmtOpts), @@ -16336,7 +16352,7 @@ yynewstate: } parser.yyVAL.item = st } - case 1465: + case 1466: { st := yyS[yypt-2].item.(*ast.SelectStmt) lastField := st.Fields.Fields[len(st.Fields.Fields)-1] @@ -16348,7 +16364,7 @@ yynewstate: st.Where = yyS[yypt-0].item.(ast.ExprNode) } } - case 1466: + case 1467: { st := yyS[yypt-6].item.(*ast.SelectStmt) st.From = yyS[yypt-4].item.(*ast.TableRefsClause) @@ -16371,11 +16387,11 @@ yynewstate: } parser.yyVAL.item = st } - case 1467: + case 1468: { parser.yyVAL.item = nil } - case 1468: + case 1469: { var repSeed ast.ExprNode if yyS[yypt-0].expr != nil { @@ -16388,7 +16404,7 @@ yynewstate: RepeatableSeed: repSeed, } } - case 1469: + case 1470: { var repSeed ast.ExprNode if yyS[yypt-0].expr != nil { @@ -16399,43 +16415,43 @@ yynewstate: RepeatableSeed: repSeed, } } - case 1470: + case 1471: { parser.yyVAL.item = ast.SampleMethodTypeNone } - case 1471: + case 1472: { parser.yyVAL.item = ast.SampleMethodTypeSystem } - case 1472: + case 1473: { parser.yyVAL.item = ast.SampleMethodTypeBernoulli } - case 1473: + case 1474: { parser.yyVAL.item = ast.SampleMethodTypeTiDBRegion } - case 1474: + case 1475: { parser.yyVAL.item = ast.SampleClauseUnitTypeDefault } - case 1475: + case 1476: { parser.yyVAL.item = ast.SampleClauseUnitTypeRow } - case 1476: + case 1477: { parser.yyVAL.item = ast.SampleClauseUnitTypePercent } - case 1477: + case 1478: { parser.yyVAL.expr = nil } - case 1478: + case 1479: { parser.yyVAL.expr = yyS[yypt-1].expr } - case 1479: + case 1480: { st := yyS[yypt-6].item.(*ast.SelectStmt) if yyS[yypt-1].item != nil { @@ -16482,7 +16498,7 @@ yynewstate: } parser.yyVAL.statement = st } - case 1480: + case 1481: { st := yyS[yypt-5].item.(*ast.SelectStmt) if yyS[yypt-4].item != nil { @@ -16502,7 +16518,7 @@ yynewstate: } parser.yyVAL.statement = st } - case 1481: + case 1482: { st := yyS[yypt-4].item.(*ast.SelectStmt) if yyS[yypt-1].item != nil { @@ -16519,7 +16535,7 @@ yynewstate: } parser.yyVAL.statement = st } - case 1482: + case 1483: { st := &ast.SelectStmt{ Kind: ast.SelectStmtKindTable, @@ -16541,7 +16557,7 @@ yynewstate: } parser.yyVAL.statement = st } - case 1483: + case 1484: { st := &ast.SelectStmt{ Kind: ast.SelectStmtKindValues, @@ -16562,13 +16578,13 @@ yynewstate: } parser.yyVAL.statement = st } - case 1484: + case 1485: { sel := yyS[yypt-0].statement.(*ast.SelectStmt) sel.With = yyS[yypt-1].item.(*ast.WithClause) parser.yyVAL.statement = sel } - case 1485: + case 1486: { var sel ast.StmtNode switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -16584,30 +16600,30 @@ yynewstate: } parser.yyVAL.statement = sel } - case 1486: + case 1487: { parser.yyVAL.item = yyS[yypt-0].item } - case 1487: + case 1488: { ws := yyS[yypt-0].item.(*ast.WithClause) ws.IsRecursive = true parser.yyVAL.item = ws } - case 1488: + case 1489: { ws := yyS[yypt-2].item.(*ast.WithClause) ws.CTEs = append(ws.CTEs, yyS[yypt-0].item.(*ast.CommonTableExpression)) parser.yyVAL.item = ws } - case 1489: + case 1490: { ws := &ast.WithClause{} ws.CTEs = make([]*ast.CommonTableExpression, 0, 4) ws.CTEs = append(ws.CTEs, yyS[yypt-0].item.(*ast.CommonTableExpression)) parser.yyVAL.item = ws } - case 1490: + case 1491: { cte := &ast.CommonTableExpression{} cte.Name = model.NewCIStr(yyS[yypt-3].ident) @@ -16615,37 +16631,37 @@ yynewstate: cte.Query = yyS[yypt-0].expr.(*ast.SubqueryExpr) parser.yyVAL.item = cte } - case 1492: + case 1493: { parser.yyVAL.item = nil } - case 1493: + case 1494: { parser.yyVAL.item = yyS[yypt-0].item.([]ast.WindowSpec) } - case 1494: + case 1495: { parser.yyVAL.item = []ast.WindowSpec{yyS[yypt-0].item.(ast.WindowSpec)} } - case 1495: + case 1496: { parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.WindowSpec), yyS[yypt-0].item.(ast.WindowSpec)) } - case 1496: + case 1497: { var spec = yyS[yypt-0].item.(ast.WindowSpec) spec.Name = yyS[yypt-2].item.(model.CIStr) parser.yyVAL.item = spec } - case 1497: + case 1498: { parser.yyVAL.item = model.NewCIStr(yyS[yypt-0].ident) } - case 1498: + case 1499: { parser.yyVAL.item = yyS[yypt-1].item.(ast.WindowSpec) } - case 1499: + case 1500: { spec := ast.WindowSpec{Ref: yyS[yypt-3].item.(model.CIStr)} if yyS[yypt-2].item != nil { @@ -16659,116 +16675,112 @@ yynewstate: } parser.yyVAL.item = spec } - case 1500: + case 1501: { parser.yyVAL.item = model.CIStr{} } - case 1502: + case 1503: { parser.yyVAL.item = nil } - case 1503: + case 1504: { parser.yyVAL.item = &ast.PartitionByClause{Items: yyS[yypt-0].item.([]*ast.ByItem)} } - case 1504: + case 1505: { parser.yyVAL.item = nil } - case 1505: + case 1506: { parser.yyVAL.item = &ast.OrderByClause{Items: yyS[yypt-0].item.([]*ast.ByItem)} } - case 1506: + case 1507: { parser.yyVAL.item = nil } - case 1507: + case 1508: { parser.yyVAL.item = &ast.FrameClause{ Type: yyS[yypt-1].item.(ast.FrameType), Extent: yyS[yypt-0].item.(ast.FrameExtent), } } - case 1508: + case 1509: { parser.yyVAL.item = ast.FrameType(ast.Rows) } - case 1509: + case 1510: { parser.yyVAL.item = ast.FrameType(ast.Ranges) } - case 1510: + case 1511: { parser.yyVAL.item = ast.FrameType(ast.Groups) } - case 1511: + case 1512: { parser.yyVAL.item = ast.FrameExtent{ Start: yyS[yypt-0].item.(ast.FrameBound), End: ast.FrameBound{Type: ast.CurrentRow}, } } - case 1513: + case 1514: { parser.yyVAL.item = ast.FrameBound{Type: ast.Preceding, UnBounded: true} } - case 1514: + case 1515: { parser.yyVAL.item = ast.FrameBound{Type: ast.Preceding, Expr: ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)} } - case 1515: + case 1516: { parser.yyVAL.item = ast.FrameBound{Type: ast.Preceding, Expr: ast.NewParamMarkerExpr(yyS[yypt].offset)} } - case 1516: + case 1517: { parser.yyVAL.item = ast.FrameBound{Type: ast.Preceding, Expr: yyS[yypt-2].expr, Unit: yyS[yypt-1].item.(ast.TimeUnitType)} } - case 1517: + case 1518: { parser.yyVAL.item = ast.FrameBound{Type: ast.CurrentRow} } - case 1518: + case 1519: { parser.yyVAL.item = ast.FrameExtent{Start: yyS[yypt-2].item.(ast.FrameBound), End: yyS[yypt-0].item.(ast.FrameBound)} } - case 1520: + case 1521: { parser.yyVAL.item = ast.FrameBound{Type: ast.Following, UnBounded: true} } - case 1521: + case 1522: { parser.yyVAL.item = ast.FrameBound{Type: ast.Following, Expr: ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)} } - case 1522: + case 1523: { parser.yyVAL.item = ast.FrameBound{Type: ast.Following, Expr: ast.NewParamMarkerExpr(yyS[yypt].offset)} } - case 1523: + case 1524: { parser.yyVAL.item = ast.FrameBound{Type: ast.Following, Expr: yyS[yypt-2].expr, Unit: yyS[yypt-1].item.(ast.TimeUnitType)} } - case 1524: + case 1525: { parser.yyVAL.item = nil } - case 1525: + case 1526: { spec := yyS[yypt-0].item.(ast.WindowSpec) parser.yyVAL.item = &spec } - case 1526: - { - parser.yyVAL.item = yyS[yypt-0].item.(ast.WindowSpec) - } case 1527: { - parser.yyVAL.item = ast.WindowSpec{Name: yyS[yypt-0].item.(model.CIStr), OnlyAlias: true} + parser.yyVAL.item = yyS[yypt-0].item.(ast.WindowSpec) } - case 1529: + case 1528: { - parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-3].ident, Spec: yyS[yypt-0].item.(ast.WindowSpec)} + parser.yyVAL.item = ast.WindowSpec{Name: yyS[yypt-0].item.(model.CIStr), OnlyAlias: true} } case 1530: { @@ -16788,9 +16800,13 @@ yynewstate: } case 1534: { - parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: yyS[yypt-0].item.(ast.WindowSpec)} + parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-3].ident, Spec: yyS[yypt-0].item.(ast.WindowSpec)} } case 1535: + { + parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: yyS[yypt-0].item.(ast.WindowSpec)} + } + case 1536: { args := []ast.ExprNode{yyS[yypt-4].expr} if yyS[yypt-3].item != nil { @@ -16798,7 +16814,7 @@ yynewstate: } parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-6].ident, Args: args, IgnoreNull: yyS[yypt-1].item.(bool), Spec: yyS[yypt-0].item.(ast.WindowSpec)} } - case 1536: + case 1537: { args := []ast.ExprNode{yyS[yypt-4].expr} if yyS[yypt-3].item != nil { @@ -16806,23 +16822,23 @@ yynewstate: } parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-6].ident, Args: args, IgnoreNull: yyS[yypt-1].item.(bool), Spec: yyS[yypt-0].item.(ast.WindowSpec)} } - case 1537: + case 1538: { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-3].expr}, IgnoreNull: yyS[yypt-1].item.(bool), Spec: yyS[yypt-0].item.(ast.WindowSpec)} } - case 1538: + case 1539: { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-3].expr}, IgnoreNull: yyS[yypt-1].item.(bool), Spec: yyS[yypt-0].item.(ast.WindowSpec)} } - case 1539: + case 1540: { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-8].ident, Args: []ast.ExprNode{yyS[yypt-6].expr, yyS[yypt-4].expr}, FromLast: yyS[yypt-2].item.(bool), IgnoreNull: yyS[yypt-1].item.(bool), Spec: yyS[yypt-0].item.(ast.WindowSpec)} } - case 1540: + case 1541: { parser.yyVAL.item = nil } - case 1541: + case 1542: { args := []ast.ExprNode{ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)} if yyS[yypt-0].item != nil { @@ -16830,7 +16846,7 @@ yynewstate: } parser.yyVAL.item = args } - case 1542: + case 1543: { args := []ast.ExprNode{ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)} if yyS[yypt-0].item != nil { @@ -16838,17 +16854,13 @@ yynewstate: } parser.yyVAL.item = args } - case 1543: - { - parser.yyVAL.item = nil - } case 1544: { - parser.yyVAL.item = yyS[yypt-0].expr + parser.yyVAL.item = nil } case 1545: { - parser.yyVAL.item = false + parser.yyVAL.item = yyS[yypt-0].expr } case 1546: { @@ -16856,11 +16868,11 @@ yynewstate: } case 1547: { - parser.yyVAL.item = true + parser.yyVAL.item = false } case 1548: { - parser.yyVAL.item = false + parser.yyVAL.item = true } case 1549: { @@ -16868,13 +16880,17 @@ yynewstate: } case 1550: { - parser.yyVAL.item = true + parser.yyVAL.item = false } case 1551: { - parser.yyVAL.item = &ast.TableRefsClause{TableRefs: yyS[yypt-0].item.(*ast.Join)} + parser.yyVAL.item = true } case 1552: + { + parser.yyVAL.item = &ast.TableRefsClause{TableRefs: yyS[yypt-0].item.(*ast.Join)} + } + case 1553: { if j, ok := yyS[yypt-0].item.(*ast.Join); ok { // if $1 is Join, use it directly @@ -16883,12 +16899,12 @@ yynewstate: parser.yyVAL.item = &ast.Join{Left: yyS[yypt-0].item.(ast.ResultSetNode), Right: nil} } } - case 1553: + case 1554: { /* from a, b is default cross join */ parser.yyVAL.item = &ast.Join{Left: yyS[yypt-2].item.(ast.ResultSetNode), Right: yyS[yypt-0].item.(ast.ResultSetNode), Tp: ast.CrossJoin} } - case 1555: + case 1556: { /* * ODBC escape syntax for outer join is { OJ join_table } @@ -16896,7 +16912,7 @@ yynewstate: */ parser.yyVAL.item = yyS[yypt-1].item } - case 1558: + case 1559: { tn := yyS[yypt-5].item.(*ast.TableName) tn.PartitionNames = yyS[yypt-4].item.([]model.CIStr) @@ -16909,66 +16925,66 @@ yynewstate: } parser.yyVAL.item = &ast.TableSource{Source: tn, AsName: yyS[yypt-3].item.(model.CIStr)} } - case 1559: + case 1560: { resultNode := yyS[yypt-1].expr.(*ast.SubqueryExpr).Query parser.yyVAL.item = &ast.TableSource{Source: resultNode, AsName: yyS[yypt-0].item.(model.CIStr)} } - case 1560: + case 1561: { j := yyS[yypt-1].item.(*ast.Join) j.ExplicitParens = true parser.yyVAL.item = yyS[yypt-1].item } - case 1561: + case 1562: { parser.yyVAL.item = []model.CIStr{} } - case 1562: + case 1563: { parser.yyVAL.item = yyS[yypt-1].item } - case 1563: + case 1564: { parser.yyVAL.item = model.CIStr{} } - case 1565: + case 1566: { parser.yyVAL.item = model.NewCIStr(yyS[yypt-0].ident) } - case 1566: + case 1567: { parser.yyVAL.item = model.NewCIStr(yyS[yypt-0].ident) } - case 1567: + case 1568: { parser.yyVAL.item = ast.HintUse } - case 1568: + case 1569: { parser.yyVAL.item = ast.HintIgnore } - case 1569: + case 1570: { parser.yyVAL.item = ast.HintForce } - case 1570: + case 1571: { parser.yyVAL.item = ast.HintForScan } - case 1571: + case 1572: { parser.yyVAL.item = ast.HintForJoin } - case 1572: + case 1573: { parser.yyVAL.item = ast.HintForOrderBy } - case 1573: + case 1574: { parser.yyVAL.item = ast.HintForGroupBy } - case 1574: + case 1575: { parser.yyVAL.item = &ast.IndexHint{ IndexNames: yyS[yypt-1].item.([]model.CIStr), @@ -16976,134 +16992,134 @@ yynewstate: HintScope: yyS[yypt-3].item.(ast.IndexHintScope), } } - case 1575: + case 1576: { var nameList []model.CIStr parser.yyVAL.item = nameList } - case 1576: + case 1577: { parser.yyVAL.item = []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)} } - case 1577: + case 1578: { parser.yyVAL.item = append(yyS[yypt-2].item.([]model.CIStr), model.NewCIStr(yyS[yypt-0].ident)) } - case 1578: + case 1579: { parser.yyVAL.item = []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)} } - case 1579: + case 1580: { parser.yyVAL.item = append(yyS[yypt-2].item.([]model.CIStr), model.NewCIStr(yyS[yypt-0].ident)) } - case 1580: + case 1581: { parser.yyVAL.item = []*ast.IndexHint{yyS[yypt-0].item.(*ast.IndexHint)} } - case 1581: + case 1582: { parser.yyVAL.item = append(yyS[yypt-1].item.([]*ast.IndexHint), yyS[yypt-0].item.(*ast.IndexHint)) } - case 1582: + case 1583: { parser.yyVAL.item = []*ast.IndexHint{} } - case 1584: + case 1585: { parser.yyVAL.item = ast.NewCrossJoin(yyS[yypt-2].item.(ast.ResultSetNode), yyS[yypt-0].item.(ast.ResultSetNode)) } - case 1585: + case 1586: { on := &ast.OnCondition{Expr: yyS[yypt-0].expr} parser.yyVAL.item = &ast.Join{Left: yyS[yypt-4].item.(ast.ResultSetNode), Right: yyS[yypt-2].item.(ast.ResultSetNode), Tp: ast.CrossJoin, On: on} } - case 1586: + case 1587: { parser.yyVAL.item = &ast.Join{Left: yyS[yypt-6].item.(ast.ResultSetNode), Right: yyS[yypt-4].item.(ast.ResultSetNode), Tp: ast.CrossJoin, Using: yyS[yypt-1].item.([]*ast.ColumnName)} } - case 1587: + case 1588: { on := &ast.OnCondition{Expr: yyS[yypt-0].expr} parser.yyVAL.item = &ast.Join{Left: yyS[yypt-6].item.(ast.ResultSetNode), Right: yyS[yypt-2].item.(ast.ResultSetNode), Tp: yyS[yypt-5].item.(ast.JoinType), On: on} } - case 1588: + case 1589: { parser.yyVAL.item = &ast.Join{Left: yyS[yypt-8].item.(ast.ResultSetNode), Right: yyS[yypt-4].item.(ast.ResultSetNode), Tp: yyS[yypt-7].item.(ast.JoinType), Using: yyS[yypt-1].item.([]*ast.ColumnName)} } - case 1589: + case 1590: { parser.yyVAL.item = &ast.Join{Left: yyS[yypt-3].item.(ast.ResultSetNode), Right: yyS[yypt-0].item.(ast.ResultSetNode), NaturalJoin: true} } - case 1590: + case 1591: { parser.yyVAL.item = &ast.Join{Left: yyS[yypt-5].item.(ast.ResultSetNode), Right: yyS[yypt-0].item.(ast.ResultSetNode), Tp: yyS[yypt-3].item.(ast.JoinType), NaturalJoin: true} } - case 1591: + case 1592: { parser.yyVAL.item = &ast.Join{Left: yyS[yypt-2].item.(ast.ResultSetNode), Right: yyS[yypt-0].item.(ast.ResultSetNode), StraightJoin: true} } - case 1592: + case 1593: { on := &ast.OnCondition{Expr: yyS[yypt-0].expr} parser.yyVAL.item = &ast.Join{Left: yyS[yypt-4].item.(ast.ResultSetNode), Right: yyS[yypt-2].item.(ast.ResultSetNode), StraightJoin: true, On: on} } - case 1593: + case 1594: { parser.yyVAL.item = ast.LeftJoin } - case 1594: + case 1595: { parser.yyVAL.item = ast.RightJoin } - case 1600: + case 1601: { parser.yyVAL.item = nil } - case 1601: + case 1602: { parser.yyVAL.item = &ast.Limit{Count: yyS[yypt-0].item.(ast.ValueExpr)} } - case 1602: + case 1603: { parser.yyVAL.item = ast.NewValueExpr(yyS[yypt-0].item, parser.charset, parser.collation) } - case 1603: + case 1604: { parser.yyVAL.item = ast.NewParamMarkerExpr(yyS[yypt].offset) } - case 1608: + case 1609: { parser.yyVAL.item = ast.NewValueExpr(uint64(1), parser.charset, parser.collation) } - case 1610: + case 1611: { parser.yyVAL.item = &ast.Limit{Count: yyS[yypt-0].item.(ast.ExprNode)} } - case 1611: + case 1612: { parser.yyVAL.item = &ast.Limit{Offset: yyS[yypt-2].item.(ast.ExprNode), Count: yyS[yypt-0].item.(ast.ExprNode)} } - case 1612: + case 1613: { parser.yyVAL.item = &ast.Limit{Offset: yyS[yypt-0].item.(ast.ExprNode), Count: yyS[yypt-2].item.(ast.ExprNode)} } - case 1613: + case 1614: { parser.yyVAL.item = &ast.Limit{Count: yyS[yypt-2].item.(ast.ExprNode)} } - case 1614: + case 1615: { parser.yyVAL.item = nil } - case 1616: + case 1617: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true opt.TableHints = yyS[yypt-0].item.([]*ast.TableOptimizerHint) parser.yyVAL.item = opt } - case 1617: + case 1618: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true @@ -17115,61 +17131,61 @@ yynewstate: } parser.yyVAL.item = opt } - case 1618: + case 1619: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true opt.Priority = yyS[yypt-0].item.(mysql.PriorityEnum) parser.yyVAL.item = opt } - case 1619: + case 1620: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true opt.SQLSmallResult = true parser.yyVAL.item = opt } - case 1620: + case 1621: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true opt.SQLBigResult = true parser.yyVAL.item = opt } - case 1621: + case 1622: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true opt.SQLBufferResult = true parser.yyVAL.item = opt } - case 1622: + case 1623: { opt := &ast.SelectStmtOpts{} opt.SQLCache = yyS[yypt-0].item.(bool) parser.yyVAL.item = opt } - case 1623: + case 1624: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true opt.CalcFoundRows = true parser.yyVAL.item = opt } - case 1624: + case 1625: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true opt.StraightJoin = true parser.yyVAL.item = opt } - case 1625: + case 1626: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true parser.yyVAL.item = opt } - case 1627: + case 1628: { opts := yyS[yypt-1].item.(*ast.SelectStmtOpts) opt := yyS[yypt-0].item.(*ast.SelectStmtOpts) @@ -17214,7 +17230,7 @@ yynewstate: parser.yyVAL.item = opts } - case 1629: + case 1630: { hints, warns := parser.parseHint(yyS[yypt-0].ident) for _, w := range warns { @@ -17223,31 +17239,31 @@ yynewstate: } parser.yyVAL.item = hints } - case 1630: + case 1631: { parser.yyVAL.item = nil } - case 1632: + case 1633: { parser.yyVAL.item = true } - case 1633: + case 1634: { parser.yyVAL.item = false } - case 1634: + case 1635: { parser.yyVAL.item = &ast.FieldList{Fields: yyS[yypt-0].item.([]*ast.SelectField)} } - case 1635: + case 1636: { parser.yyVAL.item = nil } - case 1637: + case 1638: { parser.yyVAL.item = nil } - case 1638: + case 1639: { x := &ast.SelectIntoOption{ Tp: ast.SelectIntoOutfile, @@ -17262,7 +17278,7 @@ yynewstate: parser.yyVAL.item = x } - case 1639: + case 1640: { rs := yyS[yypt-1].statement.(*ast.SelectStmt) endOffset := parser.endOffset(&yyS[yypt]) @@ -17272,14 +17288,14 @@ yynewstate: rs.SetText(src[yyS[yypt-1].offset:yyS[yypt].offset]) parser.yyVAL.expr = &ast.SubqueryExpr{Query: rs} } - case 1640: + case 1641: { rs := yyS[yypt-1].statement.(*ast.SetOprStmt) src := parser.src rs.SetText(src[yyS[yypt-1].offset:yyS[yypt].offset]) parser.yyVAL.expr = &ast.SubqueryExpr{Query: rs} } - case 1641: + case 1642: { rs := yyS[yypt-1].statement.(*ast.SelectStmt) endOffset := parser.endOffset(&yyS[yypt]) @@ -17289,32 +17305,32 @@ yynewstate: rs.SetText(src[yyS[yypt-1].offset:yyS[yypt].offset]) parser.yyVAL.expr = &ast.SubqueryExpr{Query: rs} } - case 1642: + case 1643: { parser.yyVAL.item = nil } - case 1643: + case 1644: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForUpdate, Tables: yyS[yypt-0].item.([]*ast.TableName), } } - case 1644: + case 1645: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForShare, Tables: yyS[yypt-0].item.([]*ast.TableName), } } - case 1645: + case 1646: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForUpdateNoWait, Tables: yyS[yypt-1].item.([]*ast.TableName), } } - case 1646: + case 1647: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForUpdateWaitN, @@ -17322,55 +17338,55 @@ yynewstate: Tables: yyS[yypt-2].item.([]*ast.TableName), } } - case 1647: + case 1648: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForShareNoWait, Tables: yyS[yypt-1].item.([]*ast.TableName), } } - case 1648: + case 1649: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForUpdateSkipLocked, Tables: yyS[yypt-2].item.([]*ast.TableName), } } - case 1649: + case 1650: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForShareSkipLocked, Tables: yyS[yypt-2].item.([]*ast.TableName), } } - case 1650: + case 1651: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForShare, Tables: []*ast.TableName{}, } } - case 1651: + case 1652: { parser.yyVAL.item = []*ast.TableName{} } - case 1652: + case 1653: { parser.yyVAL.item = yyS[yypt-0].item.([]*ast.TableName) } - case 1655: + case 1656: { setOpr := yyS[yypt-0].statement.(*ast.SetOprStmt) setOpr.With = yyS[yypt-1].item.(*ast.WithClause) parser.yyVAL.statement = setOpr } - case 1656: + case 1657: { setOpr := yyS[yypt-0].statement.(*ast.SetOprStmt) setOpr.With = yyS[yypt-1].item.(*ast.WithClause) parser.yyVAL.statement = setOpr } - case 1657: + case 1658: { setOprList1 := yyS[yypt-2].item.([]ast.Node) if sel, isSelect := setOprList1[len(setOprList1)-1].(*ast.SelectStmt); isSelect && !sel.IsInBraces { @@ -17387,7 +17403,7 @@ yynewstate: setOpr.SelectList.Selects = append(setOpr.SelectList.Selects, st) parser.yyVAL.statement = setOpr } - case 1658: + case 1659: { setOprList1 := yyS[yypt-2].item.([]ast.Node) if sel, isSelect := setOprList1[len(setOprList1)-1].(*ast.SelectStmt); isSelect && !sel.IsInBraces { @@ -17410,7 +17426,7 @@ yynewstate: setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}} parser.yyVAL.statement = setOpr } - case 1659: + case 1660: { setOprList1 := yyS[yypt-3].item.([]ast.Node) if sel, isSelect := setOprList1[len(setOprList1)-1].(*ast.SelectStmt); isSelect && !sel.IsInBraces { @@ -17434,7 +17450,7 @@ yynewstate: setOpr.OrderBy = yyS[yypt-0].item.(*ast.OrderByClause) parser.yyVAL.statement = setOpr } - case 1660: + case 1661: { setOprList1 := yyS[yypt-3].item.([]ast.Node) if sel, isSelect := setOprList1[len(setOprList1)-1].(*ast.SelectStmt); isSelect && !sel.IsInBraces { @@ -17458,7 +17474,7 @@ yynewstate: setOpr.Limit = yyS[yypt-0].item.(*ast.Limit) parser.yyVAL.statement = setOpr } - case 1661: + case 1662: { setOprList1 := yyS[yypt-4].item.([]ast.Node) if sel, isSelect := setOprList1[len(setOprList1)-1].(*ast.SelectStmt); isSelect && !sel.IsInBraces { @@ -17483,7 +17499,7 @@ yynewstate: setOpr.Limit = yyS[yypt-0].item.(*ast.Limit) parser.yyVAL.statement = setOpr } - case 1662: + case 1663: { var setOprList []ast.Node var with *ast.WithClause @@ -17499,7 +17515,7 @@ yynewstate: setOpr.OrderBy = yyS[yypt-0].item.(*ast.OrderByClause) parser.yyVAL.statement = setOpr } - case 1663: + case 1664: { var setOprList []ast.Node var with *ast.WithClause @@ -17515,7 +17531,7 @@ yynewstate: setOpr.Limit = yyS[yypt-0].item.(*ast.Limit) parser.yyVAL.statement = setOpr } - case 1664: + case 1665: { var setOprList []ast.Node var with *ast.WithClause @@ -17532,7 +17548,7 @@ yynewstate: setOpr.Limit = yyS[yypt-0].item.(*ast.Limit) parser.yyVAL.statement = setOpr } - case 1666: + case 1667: { setOprList1 := yyS[yypt-2].item.([]ast.Node) setOprList2 := yyS[yypt-0].item.([]ast.Node) @@ -17548,11 +17564,11 @@ yynewstate: } parser.yyVAL.item = append(setOprList1, setOprList2...) } - case 1667: + case 1668: { parser.yyVAL.item = []ast.Node{yyS[yypt-0].statement.(*ast.SelectStmt)} } - case 1668: + case 1669: { var setOprList []ast.Node switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -17563,7 +17579,7 @@ yynewstate: } parser.yyVAL.item = setOprList } - case 1669: + case 1670: { var tp ast.SetOprType tp = ast.Union @@ -17572,7 +17588,7 @@ yynewstate: } parser.yyVAL.item = &tp } - case 1670: + case 1671: { var tp ast.SetOprType tp = ast.Except @@ -17581,7 +17597,7 @@ yynewstate: } parser.yyVAL.item = &tp } - case 1671: + case 1672: { var tp ast.SetOprType tp = ast.Intersect @@ -17590,7 +17606,7 @@ yynewstate: } parser.yyVAL.item = &tp } - case 1673: + case 1674: { parser.yyVAL.statement = &ast.ChangeStmt{ NodeType: ast.PumpType, @@ -17598,7 +17614,7 @@ yynewstate: NodeID: yyS[yypt-0].ident, } } - case 1674: + case 1675: { parser.yyVAL.statement = &ast.ChangeStmt{ NodeType: ast.DrainerType, @@ -17606,19 +17622,19 @@ yynewstate: NodeID: yyS[yypt-0].ident, } } - case 1675: + case 1676: { parser.yyVAL.statement = &ast.SetStmt{Variables: yyS[yypt-0].item.([]*ast.VariableAssignment)} } - case 1676: + case 1677: { parser.yyVAL.statement = &ast.SetPwdStmt{Password: yyS[yypt-0].ident} } - case 1677: + case 1678: { parser.yyVAL.statement = &ast.SetPwdStmt{User: yyS[yypt-2].item.(*auth.UserIdentity), Password: yyS[yypt-0].ident} } - case 1678: + case 1679: { vars := yyS[yypt-0].item.([]*ast.VariableAssignment) for _, v := range vars { @@ -17626,11 +17642,11 @@ yynewstate: } parser.yyVAL.statement = &ast.SetStmt{Variables: vars} } - case 1679: + case 1680: { parser.yyVAL.statement = &ast.SetStmt{Variables: yyS[yypt-0].item.([]*ast.VariableAssignment)} } - case 1680: + case 1681: { assigns := yyS[yypt-0].item.([]*ast.VariableAssignment) for i := 0; i < len(assigns); i++ { @@ -17641,19 +17657,19 @@ yynewstate: } parser.yyVAL.statement = &ast.SetStmt{Variables: assigns} } - case 1681: + case 1682: { parser.yyVAL.statement = &ast.SetConfigStmt{Type: strings.ToLower(yyS[yypt-3].ident), Name: yyS[yypt-2].ident, Value: yyS[yypt-0].expr} } - case 1682: + case 1683: { parser.yyVAL.statement = &ast.SetConfigStmt{Instance: yyS[yypt-3].ident, Name: yyS[yypt-2].ident, Value: yyS[yypt-0].expr} } - case 1683: + case 1684: { parser.yyVAL.statement = yyS[yypt-0].item.(*ast.SetRoleStmt) } - case 1684: + case 1685: { tmp := yyS[yypt-2].item.(*ast.SetRoleStmt) parser.yyVAL.statement = &ast.SetDefaultRoleStmt{ @@ -17662,27 +17678,27 @@ yynewstate: UserList: yyS[yypt-0].item.([]*auth.UserIdentity), } } - case 1685: + case 1686: { parser.yyVAL.item = &ast.SetRoleStmt{SetRoleOpt: ast.SetRoleNone, RoleList: nil} } - case 1686: + case 1687: { parser.yyVAL.item = &ast.SetRoleStmt{SetRoleOpt: ast.SetRoleAll, RoleList: nil} } - case 1687: + case 1688: { parser.yyVAL.item = &ast.SetRoleStmt{SetRoleOpt: ast.SetRoleRegular, RoleList: yyS[yypt-0].item.([]*auth.RoleIdentity)} } - case 1688: + case 1689: { parser.yyVAL.item = &ast.SetRoleStmt{SetRoleOpt: ast.SetRoleAllExcept, RoleList: yyS[yypt-0].item.([]*auth.RoleIdentity)} } - case 1690: + case 1691: { parser.yyVAL.item = &ast.SetRoleStmt{SetRoleOpt: ast.SetRoleDefault, RoleList: nil} } - case 1691: + case 1692: { if yyS[yypt-0].item != nil { parser.yyVAL.item = yyS[yypt-0].item @@ -17690,7 +17706,7 @@ yynewstate: parser.yyVAL.item = []*ast.VariableAssignment{} } } - case 1692: + case 1693: { if yyS[yypt-0].item != nil { varAssigns := yyS[yypt-0].item.([]*ast.VariableAssignment) @@ -17699,28 +17715,28 @@ yynewstate: parser.yyVAL.item = yyS[yypt-2].item } } - case 1693: + case 1694: { varAssigns := []*ast.VariableAssignment{} expr := ast.NewValueExpr(yyS[yypt-0].ident, parser.charset, parser.collation) varAssigns = append(varAssigns, &ast.VariableAssignment{Name: "tx_isolation", Value: expr, IsSystem: true}) parser.yyVAL.item = varAssigns } - case 1694: + case 1695: { varAssigns := []*ast.VariableAssignment{} expr := ast.NewValueExpr("0", parser.charset, parser.collation) varAssigns = append(varAssigns, &ast.VariableAssignment{Name: "tx_read_only", Value: expr, IsSystem: true}) parser.yyVAL.item = varAssigns } - case 1695: + case 1696: { varAssigns := []*ast.VariableAssignment{} expr := ast.NewValueExpr("1", parser.charset, parser.collation) varAssigns = append(varAssigns, &ast.VariableAssignment{Name: "tx_read_only", Value: expr, IsSystem: true}) parser.yyVAL.item = varAssigns } - case 1696: + case 1697: { varAssigns := []*ast.VariableAssignment{} asof := yyS[yypt-0].item.(*ast.AsOfClause) @@ -17729,59 +17745,59 @@ yynewstate: } parser.yyVAL.item = varAssigns } - case 1697: + case 1698: { parser.yyVAL.ident = ast.RepeatableRead } - case 1698: + case 1699: { parser.yyVAL.ident = ast.ReadCommitted } - case 1699: + case 1700: { parser.yyVAL.ident = ast.ReadUncommitted } - case 1700: + case 1701: { parser.yyVAL.ident = ast.Serializable } - case 1701: + case 1702: { parser.yyVAL.expr = ast.NewValueExpr("ON", parser.charset, parser.collation) } - case 1702: + case 1703: { parser.yyVAL.expr = ast.NewValueExpr("BINARY", parser.charset, parser.collation) } - case 1707: + case 1708: { parser.yyVAL.ident = yyS[yypt-2].ident + "." + yyS[yypt-0].ident } - case 1709: + case 1710: { parser.yyVAL.ident = yyS[yypt-2].ident + "." + yyS[yypt-0].ident } - case 1710: + case 1711: { parser.yyVAL.ident = yyS[yypt-2].ident + "-" + yyS[yypt-0].ident } - case 1711: + case 1712: { parser.yyVAL.item = &ast.VariableAssignment{Name: yyS[yypt-2].ident, Value: yyS[yypt-0].expr, IsSystem: true} } - case 1712: + case 1713: { parser.yyVAL.item = &ast.VariableAssignment{Name: yyS[yypt-2].ident, Value: yyS[yypt-0].expr, IsGlobal: true, IsSystem: true} } - case 1713: + case 1714: { parser.yyVAL.item = &ast.VariableAssignment{Name: yyS[yypt-2].ident, Value: yyS[yypt-0].expr, IsSystem: true} } - case 1714: + case 1715: { parser.yyVAL.item = &ast.VariableAssignment{Name: yyS[yypt-2].ident, Value: yyS[yypt-0].expr, IsSystem: true} } - case 1715: + case 1716: { v := strings.ToLower(yyS[yypt-2].ident) var isGlobal bool @@ -17797,27 +17813,27 @@ yynewstate: } parser.yyVAL.item = &ast.VariableAssignment{Name: v, Value: yyS[yypt-0].expr, IsGlobal: isGlobal, IsSystem: true} } - case 1716: + case 1717: { v := yyS[yypt-2].ident v = strings.TrimPrefix(v, "@") parser.yyVAL.item = &ast.VariableAssignment{Name: v, Value: yyS[yypt-0].expr} } - case 1717: + case 1718: { parser.yyVAL.item = &ast.VariableAssignment{ Name: ast.SetNames, Value: ast.NewValueExpr(yyS[yypt-0].ident, "", ""), } } - case 1718: + case 1719: { parser.yyVAL.item = &ast.VariableAssignment{ Name: ast.SetNames, Value: ast.NewValueExpr(yyS[yypt-2].ident, "", ""), } } - case 1719: + case 1720: { parser.yyVAL.item = &ast.VariableAssignment{ Name: ast.SetNames, @@ -17825,24 +17841,24 @@ yynewstate: ExtendValue: ast.NewValueExpr(yyS[yypt-0].ident, "", ""), } } - case 1720: + case 1721: { v := &ast.DefaultExpr{} parser.yyVAL.item = &ast.VariableAssignment{Name: ast.SetNames, Value: v} } - case 1721: + case 1722: { parser.yyVAL.item = &ast.VariableAssignment{Name: ast.SetCharset, Value: yyS[yypt-0].expr} } - case 1722: + case 1723: { parser.yyVAL.expr = ast.NewValueExpr(yyS[yypt-0].ident, "", "") } - case 1723: + case 1724: { parser.yyVAL.expr = &ast.DefaultExpr{} } - case 1724: + case 1725: { // Validate input charset name to keep the same behavior as parser of MySQL. cs, err := charset.GetCharsetInfo(yyS[yypt-0].ident) @@ -17854,11 +17870,11 @@ yynewstate: // to keep lower case of input for generated column restore. parser.yyVAL.ident = cs.Name } - case 1725: + case 1726: { parser.yyVAL.ident = charset.CharsetBin } - case 1726: + case 1727: { info, err := charset.GetCollationByName(yyS[yypt-0].ident) if err != nil { @@ -17867,19 +17883,19 @@ yynewstate: } parser.yyVAL.ident = info.Name } - case 1727: + case 1728: { parser.yyVAL.ident = charset.CollationBin } - case 1728: + case 1729: { parser.yyVAL.item = []*ast.VariableAssignment{yyS[yypt-0].item.(*ast.VariableAssignment)} } - case 1729: + case 1730: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*ast.VariableAssignment), yyS[yypt-0].item.(*ast.VariableAssignment)) } - case 1732: + case 1733: { v := strings.ToLower(yyS[yypt-0].ident) var isGlobal bool @@ -17896,77 +17912,77 @@ yynewstate: } parser.yyVAL.expr = &ast.VariableExpr{Name: v, IsGlobal: isGlobal, IsSystem: true, ExplicitScope: explicitScope} } - case 1733: + case 1734: { v := yyS[yypt-0].ident v = strings.TrimPrefix(v, "@") parser.yyVAL.expr = &ast.VariableExpr{Name: v, IsGlobal: false, IsSystem: false} } - case 1734: + case 1735: { parser.yyVAL.item = &auth.UserIdentity{Username: yyS[yypt-0].ident, Hostname: "%"} } - case 1735: + case 1736: { parser.yyVAL.item = &auth.UserIdentity{Username: yyS[yypt-2].ident, Hostname: yyS[yypt-0].ident} } - case 1736: + case 1737: { parser.yyVAL.item = &auth.UserIdentity{Username: yyS[yypt-1].ident, Hostname: strings.TrimPrefix(yyS[yypt-0].ident, "@")} } - case 1737: + case 1738: { parser.yyVAL.item = &auth.UserIdentity{CurrentUser: true} } - case 1738: + case 1739: { parser.yyVAL.item = []*auth.UserIdentity{yyS[yypt-0].item.(*auth.UserIdentity)} } - case 1739: + case 1740: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*auth.UserIdentity), yyS[yypt-0].item.(*auth.UserIdentity)) } - case 1741: + case 1742: { parser.yyVAL.ident = yyS[yypt-1].ident } - case 1745: + case 1746: { parser.yyVAL.item = &auth.RoleIdentity{Username: yyS[yypt-2].ident, Hostname: yyS[yypt-0].ident} } - case 1746: + case 1747: { parser.yyVAL.item = &auth.RoleIdentity{Username: yyS[yypt-1].ident, Hostname: strings.TrimPrefix(yyS[yypt-0].ident, "@")} } - case 1747: + case 1748: { parser.yyVAL.item = &auth.RoleIdentity{Username: yyS[yypt-0].ident, Hostname: "%"} } - case 1748: + case 1749: { parser.yyVAL.item = yyS[yypt-0].item } - case 1749: + case 1750: { parser.yyVAL.item = &auth.RoleIdentity{Username: yyS[yypt-0].ident, Hostname: "%"} } - case 1750: + case 1751: { parser.yyVAL.item = yyS[yypt-0].item } - case 1751: + case 1752: { parser.yyVAL.item = []*auth.RoleIdentity{yyS[yypt-0].item.(*auth.RoleIdentity)} } - case 1752: + case 1753: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*auth.RoleIdentity), yyS[yypt-0].item.(*auth.RoleIdentity)) } - case 1753: + case 1754: { parser.yyVAL.statement = &ast.AdminStmt{Tp: ast.AdminShowDDL} } - case 1754: + case 1755: { stmt := &ast.AdminStmt{Tp: ast.AdminShowDDLJobs} if yyS[yypt-0].item != nil { @@ -17974,7 +17990,7 @@ yynewstate: } parser.yyVAL.statement = stmt } - case 1755: + case 1756: { stmt := &ast.AdminStmt{ Tp: ast.AdminShowDDLJobs, @@ -17985,21 +18001,21 @@ yynewstate: } parser.yyVAL.statement = stmt } - case 1756: + case 1757: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminShowNextRowID, Tables: []*ast.TableName{yyS[yypt-1].item.(*ast.TableName)}, } } - case 1757: + case 1758: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminCheckTable, Tables: yyS[yypt-0].item.([]*ast.TableName), } } - case 1758: + case 1759: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminCheckIndex, @@ -18007,7 +18023,7 @@ yynewstate: Index: string(yyS[yypt-0].ident), } } - case 1759: + case 1760: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminRecoverIndex, @@ -18015,7 +18031,7 @@ yynewstate: Index: string(yyS[yypt-0].ident), } } - case 1760: + case 1761: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminCleanupIndex, @@ -18023,7 +18039,7 @@ yynewstate: Index: string(yyS[yypt-0].ident), } } - case 1761: + case 1762: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminCheckIndexRange, @@ -18032,129 +18048,129 @@ yynewstate: HandleRanges: yyS[yypt-0].item.([]ast.HandleRange), } } - case 1762: + case 1763: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminChecksumTable, Tables: yyS[yypt-0].item.([]*ast.TableName), } } - case 1763: + case 1764: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminCancelDDLJobs, JobIDs: yyS[yypt-0].item.([]int64), } } - case 1764: + case 1765: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminShowDDLJobQueries, JobIDs: yyS[yypt-0].item.([]int64), } } - case 1765: + case 1766: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminShowSlow, ShowSlow: yyS[yypt-0].item.(*ast.ShowSlow), } } - case 1766: + case 1767: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminReloadExprPushdownBlacklist, } } - case 1767: + case 1768: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminReloadOptRuleBlacklist, } } - case 1768: + case 1769: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminPluginEnable, Plugins: yyS[yypt-0].item.([]string), } } - case 1769: + case 1770: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminPluginDisable, Plugins: yyS[yypt-0].item.([]string), } } - case 1770: + case 1771: { parser.yyVAL.statement = &ast.CleanupTableLockStmt{ Tables: yyS[yypt-0].item.([]*ast.TableName), } } - case 1771: + case 1772: { parser.yyVAL.statement = &ast.RepairTableStmt{ Table: yyS[yypt-1].item.(*ast.TableName), CreateStmt: yyS[yypt-0].statement.(*ast.CreateTableStmt), } } - case 1772: + case 1773: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminFlushBindings, } } - case 1773: + case 1774: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminCaptureBindings, } } - case 1774: + case 1775: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminEvolveBindings, } } - case 1775: + case 1776: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminReloadBindings, } } - case 1776: + case 1777: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminReloadStatistics, } } - case 1777: + case 1778: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminReloadStatistics, } } - case 1778: + case 1779: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminShowTelemetry, } } - case 1779: + case 1780: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminResetTelemetryID, } } - case 1780: + case 1781: { parser.yyVAL.item = &ast.ShowSlow{ Tp: ast.ShowSlowRecent, Count: getUint64FromNUM(yyS[yypt-0].item), } } - case 1781: + case 1782: { parser.yyVAL.item = &ast.ShowSlow{ Tp: ast.ShowSlowTop, @@ -18162,7 +18178,7 @@ yynewstate: Count: getUint64FromNUM(yyS[yypt-0].item), } } - case 1782: + case 1783: { parser.yyVAL.item = &ast.ShowSlow{ Tp: ast.ShowSlowTop, @@ -18170,7 +18186,7 @@ yynewstate: Count: getUint64FromNUM(yyS[yypt-0].item), } } - case 1783: + case 1784: { parser.yyVAL.item = &ast.ShowSlow{ Tp: ast.ShowSlowTop, @@ -18178,27 +18194,27 @@ yynewstate: Count: getUint64FromNUM(yyS[yypt-0].item), } } - case 1784: + case 1785: { parser.yyVAL.item = []ast.HandleRange{yyS[yypt-0].item.(ast.HandleRange)} } - case 1785: + case 1786: { parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.HandleRange), yyS[yypt-0].item.(ast.HandleRange)) } - case 1786: + case 1787: { parser.yyVAL.item = ast.HandleRange{Begin: yyS[yypt-3].item.(int64), End: yyS[yypt-1].item.(int64)} } - case 1787: + case 1788: { parser.yyVAL.item = []int64{yyS[yypt-0].item.(int64)} } - case 1788: + case 1789: { parser.yyVAL.item = append(yyS[yypt-2].item.([]int64), yyS[yypt-0].item.(int64)) } - case 1789: + case 1790: { stmt := yyS[yypt-1].item.(*ast.ShowStmt) if yyS[yypt-0].item != nil { @@ -18210,21 +18226,21 @@ yynewstate: } parser.yyVAL.statement = stmt } - case 1790: + case 1791: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowCreateTable, Table: yyS[yypt-0].item.(*ast.TableName), } } - case 1791: + case 1792: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowCreateView, Table: yyS[yypt-0].item.(*ast.TableName), } } - case 1792: + case 1793: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowCreateDatabase, @@ -18232,21 +18248,21 @@ yynewstate: DBName: yyS[yypt-0].ident, } } - case 1793: + case 1794: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowCreateSequence, Table: yyS[yypt-0].item.(*ast.TableName), } } - case 1794: + case 1795: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowCreatePlacementPolicy, DBName: yyS[yypt-0].ident, } } - case 1795: + case 1796: { // See https://dev.mysql.com/doc/refman/5.7/en/show-create-user.html parser.yyVAL.statement = &ast.ShowStmt{ @@ -18254,14 +18270,14 @@ yynewstate: User: yyS[yypt-0].item.(*auth.UserIdentity), } } - case 1796: + case 1797: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowCreateImport, DBName: yyS[yypt-0].ident, // we reuse DBName of ShowStmt } } - case 1797: + case 1798: { stmt := &ast.ShowStmt{ Tp: ast.ShowRegions, @@ -18273,14 +18289,14 @@ yynewstate: } parser.yyVAL.statement = stmt } - case 1798: + case 1799: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowTableNextRowId, Table: yyS[yypt-1].item.(*ast.TableName), } } - case 1799: + case 1800: { stmt := &ast.ShowStmt{ Tp: ast.ShowRegions, @@ -18293,12 +18309,12 @@ yynewstate: } parser.yyVAL.statement = stmt } - case 1800: + case 1801: { // See https://dev.mysql.com/doc/refman/5.7/en/show-grants.html parser.yyVAL.statement = &ast.ShowStmt{Tp: ast.ShowGrants} } - case 1801: + case 1802: { // See https://dev.mysql.com/doc/refman/5.7/en/show-grants.html if yyS[yypt-0].item != nil { @@ -18315,26 +18331,26 @@ yynewstate: } } } - case 1802: + case 1803: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowMasterStatus, } } - case 1803: + case 1804: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowProcessList, Full: yyS[yypt-1].item.(bool), } } - case 1804: + case 1805: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowProfiles, } } - case 1805: + case 1806: { v := &ast.ShowStmt{ Tp: ast.ShowProfile, @@ -18350,37 +18366,37 @@ yynewstate: } parser.yyVAL.statement = v } - case 1806: + case 1807: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowPrivileges, } } - case 1807: + case 1808: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowBuiltins, } } - case 1808: + case 1809: { parser.yyVAL.statement = yyS[yypt-0].item.(*ast.ShowStmt) } - case 1809: + case 1810: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowPlacementForDatabase, DBName: yyS[yypt-0].ident, } } - case 1810: + case 1811: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowPlacementForTable, Table: yyS[yypt-0].item.(*ast.TableName), } } - case 1811: + case 1812: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowPlacementForPartition, @@ -18388,90 +18404,90 @@ yynewstate: Partition: model.NewCIStr(yyS[yypt-0].ident), } } - case 1812: + case 1813: { parser.yyVAL.item = nil } - case 1814: + case 1815: { parser.yyVAL.item = []int{yyS[yypt-0].item.(int)} } - case 1815: + case 1816: { l := yyS[yypt-2].item.([]int) l = append(l, yyS[yypt-0].item.(int)) parser.yyVAL.item = l } - case 1816: + case 1817: { parser.yyVAL.item = ast.ProfileTypeCPU } - case 1817: + case 1818: { parser.yyVAL.item = ast.ProfileTypeMemory } - case 1818: + case 1819: { parser.yyVAL.item = ast.ProfileTypeBlockIo } - case 1819: + case 1820: { parser.yyVAL.item = ast.ProfileTypeContextSwitch } - case 1820: + case 1821: { parser.yyVAL.item = ast.ProfileTypePageFaults } - case 1821: + case 1822: { parser.yyVAL.item = ast.ProfileTypeIpc } - case 1822: + case 1823: { parser.yyVAL.item = ast.ProfileTypeSwaps } - case 1823: + case 1824: { parser.yyVAL.item = ast.ProfileTypeSource } - case 1824: + case 1825: { parser.yyVAL.item = ast.ProfileTypeAll } - case 1825: + case 1826: { parser.yyVAL.item = nil } - case 1826: + case 1827: { v := yyS[yypt-0].item.(int64) parser.yyVAL.item = &v } - case 1827: + case 1828: { parser.yyVAL.item = nil } - case 1828: + case 1829: { parser.yyVAL.item = yyS[yypt-0].item.([]*auth.RoleIdentity) } - case 1834: + case 1835: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowEngines} } - case 1835: + case 1836: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowDatabases} } - case 1836: + case 1837: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowConfig} } - case 1837: + case 1838: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowCharset} } - case 1838: + case 1839: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowTables, @@ -18479,28 +18495,28 @@ yynewstate: Full: yyS[yypt-2].item.(bool), } } - case 1839: + case 1840: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowOpenTables, DBName: yyS[yypt-0].ident, } } - case 1840: + case 1841: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowTableStatus, DBName: yyS[yypt-0].ident, } } - case 1841: + case 1842: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowIndex, Table: yyS[yypt-0].item.(*ast.TableName), } } - case 1842: + case 1843: { show := &ast.ShowStmt{ Tp: ast.ShowIndex, @@ -18508,7 +18524,7 @@ yynewstate: } parser.yyVAL.item = show } - case 1843: + case 1844: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowColumns, @@ -18517,7 +18533,7 @@ yynewstate: Full: yyS[yypt-3].item.(bool), } } - case 1844: + case 1845: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowColumns, @@ -18527,67 +18543,67 @@ yynewstate: Extended: true, } } - case 1845: + case 1846: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowWarnings} } - case 1846: + case 1847: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowErrors} } - case 1847: + case 1848: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowVariables, GlobalScope: yyS[yypt-1].item.(bool), } } - case 1848: + case 1849: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowStatus, GlobalScope: yyS[yypt-1].item.(bool), } } - case 1849: + case 1850: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowBindings, GlobalScope: yyS[yypt-1].item.(bool), } } - case 1850: + case 1851: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowCollation, } } - case 1851: + case 1852: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowTriggers, DBName: yyS[yypt-0].ident, } } - case 1852: + case 1853: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowProcedureStatus, } } - case 1853: + case 1854: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowPumpStatus, } } - case 1854: + case 1855: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowDrainerStatus, } } - case 1855: + case 1856: { // This statement is similar to SHOW PROCEDURE STATUS but for stored functions. // See http://dev.mysql.com/doc/refman/5.7/en/show-function-status.html @@ -18597,165 +18613,165 @@ yynewstate: Tp: ast.ShowProcedureStatus, } } - case 1856: + case 1857: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowEvents, DBName: yyS[yypt-0].ident, } } - case 1857: + case 1858: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowPlugins, } } - case 1858: + case 1859: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsExtended} } - case 1859: + case 1860: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsMeta} } - case 1860: + case 1861: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsHistograms} } - case 1861: + case 1862: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsTopN} } - case 1862: + case 1863: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsBuckets} } - case 1863: + case 1864: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsHealthy} } - case 1864: + case 1865: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowColumnStatsUsage} } - case 1865: + case 1866: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowAnalyzeStatus} } - case 1866: + case 1867: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowBackups} } - case 1867: + case 1868: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowRestores} } - case 1868: + case 1869: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowImports} } - case 1869: + case 1870: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowPlacement} } - case 1870: + case 1871: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowPlacementLabels} } - case 1871: + case 1872: { parser.yyVAL.item = nil } - case 1872: + case 1873: { parser.yyVAL.item = &ast.PatternLikeExpr{ Pattern: yyS[yypt-0].expr, Escape: '\\', } } - case 1873: + case 1874: { parser.yyVAL.item = yyS[yypt-0].expr } - case 1874: + case 1875: { parser.yyVAL.item = false } - case 1875: + case 1876: { parser.yyVAL.item = true } - case 1876: + case 1877: { parser.yyVAL.item = false } - case 1877: + case 1878: { parser.yyVAL.item = false } - case 1878: + case 1879: { parser.yyVAL.item = true } - case 1879: + case 1880: { parser.yyVAL.ident = "" } - case 1880: + case 1881: { parser.yyVAL.ident = yyS[yypt-0].ident } - case 1881: + case 1882: { parser.yyVAL.item = yyS[yypt-0].item.(*ast.TableName) } - case 1882: + case 1883: { tmp := yyS[yypt-0].item.(*ast.FlushStmt) tmp.NoWriteToBinLog = yyS[yypt-1].item.(bool) parser.yyVAL.statement = tmp } - case 1883: + case 1884: { parser.yyVAL.item = []string{yyS[yypt-0].ident} } - case 1884: + case 1885: { parser.yyVAL.item = append(yyS[yypt-2].item.([]string), yyS[yypt-0].ident) } - case 1885: + case 1886: { parser.yyVAL.item = &ast.FlushStmt{ Tp: ast.FlushPrivileges, } } - case 1886: + case 1887: { parser.yyVAL.item = &ast.FlushStmt{ Tp: ast.FlushStatus, } } - case 1887: + case 1888: { parser.yyVAL.item = &ast.FlushStmt{ Tp: ast.FlushTiDBPlugin, Plugins: yyS[yypt-0].item.([]string), } } - case 1888: + case 1889: { parser.yyVAL.item = &ast.FlushStmt{ Tp: ast.FlushHosts, } } - case 1889: + case 1890: { parser.yyVAL.item = &ast.FlushStmt{ Tp: ast.FlushLogs, LogType: yyS[yypt-1].item.(ast.LogType), } } - case 1890: + case 1891: { parser.yyVAL.item = &ast.FlushStmt{ Tp: ast.FlushTables, @@ -18763,69 +18779,69 @@ yynewstate: ReadLock: yyS[yypt-0].item.(bool), } } - case 1891: + case 1892: { parser.yyVAL.item = &ast.FlushStmt{ Tp: ast.FlushClientErrorsSummary, } } - case 1892: + case 1893: { parser.yyVAL.item = ast.LogTypeDefault } - case 1893: + case 1894: { parser.yyVAL.item = ast.LogTypeBinary } - case 1894: + case 1895: { parser.yyVAL.item = ast.LogTypeEngine } - case 1895: + case 1896: { parser.yyVAL.item = ast.LogTypeError } - case 1896: + case 1897: { parser.yyVAL.item = ast.LogTypeGeneral } - case 1897: + case 1898: { parser.yyVAL.item = ast.LogTypeSlow } - case 1898: + case 1899: { parser.yyVAL.item = false } - case 1899: + case 1900: { parser.yyVAL.item = true } - case 1900: + case 1901: { parser.yyVAL.item = true } - case 1901: + case 1902: { parser.yyVAL.item = []*ast.TableName{} } - case 1903: + case 1904: { parser.yyVAL.item = []*ast.TableName{} } - case 1904: + case 1905: { parser.yyVAL.item = yyS[yypt-0].item } - case 1905: + case 1906: { parser.yyVAL.item = false } - case 1906: + case 1907: { parser.yyVAL.item = true } - case 1975: + case 1976: { var sel ast.StmtNode switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -18838,7 +18854,7 @@ yynewstate: } parser.yyVAL.statement = sel } - case 1999: + case 2000: { var sel ast.StmtNode switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -18851,7 +18867,7 @@ yynewstate: } parser.yyVAL.statement = sel } - case 2012: + case 2013: { var sel ast.StmtNode switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -18864,7 +18880,7 @@ yynewstate: } parser.yyVAL.statement = sel } - case 2014: + case 2015: { if yyS[yypt-0].statement != nil { s := yyS[yypt-0].statement @@ -18874,7 +18890,7 @@ yynewstate: parser.result = append(parser.result, s) } } - case 2015: + case 2016: { if yyS[yypt-0].statement != nil { s := yyS[yypt-0].statement @@ -18884,7 +18900,7 @@ yynewstate: parser.result = append(parser.result, s) } } - case 2016: + case 2017: { cst := yyS[yypt-0].item.(*ast.Constraint) if yyS[yypt-1].item != nil { @@ -18892,7 +18908,7 @@ yynewstate: } parser.yyVAL.item = cst } - case 2021: + case 2022: { if yyS[yypt-0].item != nil { parser.yyVAL.item = []interface{}{yyS[yypt-0].item.(interface{})} @@ -18900,7 +18916,7 @@ yynewstate: parser.yyVAL.item = []interface{}{} } } - case 2022: + case 2023: { if yyS[yypt-0].item != nil { parser.yyVAL.item = append(yyS[yypt-2].item.([]interface{}), yyS[yypt-0].item) @@ -18908,7 +18924,7 @@ yynewstate: parser.yyVAL.item = yyS[yypt-2].item } } - case 2023: + case 2024: { var columnDefs []*ast.ColumnDef var constraints []*ast.Constraint @@ -18917,7 +18933,7 @@ yynewstate: Constraints: constraints, } } - case 2024: + case 2025: { tes := yyS[yypt-1].item.([]interface{}) var columnDefs []*ast.ColumnDef @@ -18935,69 +18951,69 @@ yynewstate: Constraints: constraints, } } - case 2026: + case 2027: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionCharset, StrValue: yyS[yypt-0].ident, UintValue: ast.TableOptionCharsetWithoutConvertTo} } - case 2027: + case 2028: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionCollate, StrValue: yyS[yypt-0].ident, UintValue: ast.TableOptionCharsetWithoutConvertTo} } - case 2028: + case 2029: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionAutoIncrement, UintValue: yyS[yypt-0].item.(uint64), BoolValue: yyS[yypt-3].item.(bool)} } - case 2029: + case 2030: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionAutoIdCache, UintValue: yyS[yypt-0].item.(uint64)} } - case 2030: + case 2031: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionAutoRandomBase, UintValue: yyS[yypt-0].item.(uint64), BoolValue: yyS[yypt-3].item.(bool)} } - case 2031: + case 2032: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionAvgRowLength, UintValue: yyS[yypt-0].item.(uint64)} } - case 2032: + case 2033: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionConnection, StrValue: yyS[yypt-0].ident} } - case 2033: + case 2034: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionCheckSum, UintValue: yyS[yypt-0].item.(uint64)} } - case 2034: + case 2035: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionTableCheckSum, UintValue: yyS[yypt-0].item.(uint64)} } - case 2035: + case 2036: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionPassword, StrValue: yyS[yypt-0].ident} } - case 2036: + case 2037: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionCompression, StrValue: yyS[yypt-0].ident} } - case 2037: + case 2038: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionKeyBlockSize, UintValue: yyS[yypt-0].item.(uint64)} } - case 2038: + case 2039: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionDelayKeyWrite, UintValue: yyS[yypt-0].item.(uint64)} } - case 2039: + case 2040: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionRowFormat, UintValue: yyS[yypt-0].item.(uint64)} } - case 2040: + case 2041: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionStatsPersistent} } - case 2041: + case 2042: { n := yyS[yypt-0].item.(uint64) if n != 0 && n != 1 { @@ -19008,13 +19024,13 @@ yynewstate: yylex.AppendError(yylex.Errorf("The STATS_AUTO_RECALC is parsed but ignored by all storage engines.")) parser.lastErrorAsWarn() } - case 2042: + case 2043: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionStatsAutoRecalc, Default: true} yylex.AppendError(yylex.Errorf("The STATS_AUTO_RECALC is parsed but ignored by all storage engines.")) parser.lastErrorAsWarn() } - case 2043: + case 2044: { // Parse it but will ignore it. // In MySQL, STATS_SAMPLE_PAGES=N(Where 0 1 { @@ -20681,7 +20697,7 @@ yynewstate: OptEnclosed: true, } } - case 2362: + case 2363: { str := yyS[yypt-0].ident if str != "\\" && len(str) > 1 { @@ -20693,7 +20709,7 @@ yynewstate: Value: str, } } - case 2363: + case 2364: { str := yyS[yypt-0].ident if str != "\\" && len(str) > 1 { @@ -20705,118 +20721,118 @@ yynewstate: Value: str, } } - case 2365: + case 2366: { parser.yyVAL.ident = yyS[yypt-0].item.(ast.BinaryLiteral).ToString() } - case 2366: + case 2367: { parser.yyVAL.ident = yyS[yypt-0].item.(ast.BinaryLiteral).ToString() } - case 2367: + case 2368: { parser.yyVAL.item = &ast.LinesClause{Terminated: "\n"} } - case 2368: + case 2369: { parser.yyVAL.item = &ast.LinesClause{Starting: yyS[yypt-1].ident, Terminated: yyS[yypt-0].ident} } - case 2369: + case 2370: { parser.yyVAL.ident = "" } - case 2370: + case 2371: { parser.yyVAL.ident = yyS[yypt-0].ident } - case 2371: + case 2372: { parser.yyVAL.ident = "\n" } - case 2372: + case 2373: { parser.yyVAL.ident = yyS[yypt-0].ident } - case 2373: + case 2374: { parser.yyVAL.item = nil } - case 2374: + case 2375: { parser.yyVAL.item = yyS[yypt-0].item } - case 2375: + case 2376: { l := yyS[yypt-2].item.([]*ast.Assignment) parser.yyVAL.item = append(l, yyS[yypt-0].item.(*ast.Assignment)) } - case 2376: + case 2377: { parser.yyVAL.item = []*ast.Assignment{yyS[yypt-0].item.(*ast.Assignment)} } - case 2377: + case 2378: { parser.yyVAL.item = &ast.Assignment{ Column: yyS[yypt-2].expr.(*ast.ColumnNameExpr).Name, Expr: yyS[yypt-0].expr, } } - case 2378: + case 2379: { parser.yyVAL.statement = &ast.UnlockTablesStmt{} } - case 2379: + case 2380: { parser.yyVAL.statement = &ast.LockTablesStmt{ TableLocks: yyS[yypt-0].item.([]ast.TableLock), } } - case 2382: + case 2383: { parser.yyVAL.item = ast.TableLock{ Table: yyS[yypt-1].item.(*ast.TableName), Type: yyS[yypt-0].item.(model.TableLockType), } } - case 2383: + case 2384: { parser.yyVAL.item = model.TableLockRead } - case 2384: + case 2385: { parser.yyVAL.item = model.TableLockReadLocal } - case 2385: + case 2386: { parser.yyVAL.item = model.TableLockWrite } - case 2386: + case 2387: { parser.yyVAL.item = model.TableLockWriteLocal } - case 2387: + case 2388: { parser.yyVAL.item = []ast.TableLock{yyS[yypt-0].item.(ast.TableLock)} } - case 2388: + case 2389: { parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.TableLock), yyS[yypt-0].item.(ast.TableLock)) } - case 2389: + case 2390: { parser.yyVAL.statement = &ast.KillStmt{ ConnectionID: getUint64FromNUM(yyS[yypt-0].item), TiDBExtension: yyS[yypt-1].item.(bool), } } - case 2390: + case 2391: { parser.yyVAL.statement = &ast.KillStmt{ ConnectionID: getUint64FromNUM(yyS[yypt-0].item), TiDBExtension: yyS[yypt-2].item.(bool), } } - case 2391: + case 2392: { parser.yyVAL.statement = &ast.KillStmt{ ConnectionID: getUint64FromNUM(yyS[yypt-0].item), @@ -20824,36 +20840,37 @@ yynewstate: TiDBExtension: yyS[yypt-2].item.(bool), } } - case 2392: + case 2393: { parser.yyVAL.item = false } - case 2393: + case 2394: { parser.yyVAL.item = true } - case 2394: + case 2395: { parser.yyVAL.statement = &ast.LoadStatsStmt{ Path: yyS[yypt-0].ident, } } - case 2395: + case 2396: { parser.yyVAL.statement = &ast.DropPlacementPolicyStmt{ IfExists: yyS[yypt-1].item.(bool), PolicyName: model.NewCIStr(yyS[yypt-0].ident), } } - case 2396: + case 2397: { parser.yyVAL.statement = &ast.CreatePlacementPolicyStmt{ + OrReplace: yyS[yypt-5].item.(bool), IfNotExists: yyS[yypt-2].item.(bool), PolicyName: model.NewCIStr(yyS[yypt-1].ident), PlacementOptions: yyS[yypt-0].item.([]*ast.PlacementOption), } } - case 2397: + case 2398: { parser.yyVAL.statement = &ast.AlterPlacementPolicyStmt{ IfExists: yyS[yypt-2].item.(bool), @@ -20861,7 +20878,7 @@ yynewstate: PlacementOptions: yyS[yypt-0].item.([]*ast.PlacementOption), } } - case 2398: + case 2399: { parser.yyVAL.statement = &ast.CreateSequenceStmt{ IfNotExists: yyS[yypt-3].item.(bool), @@ -20870,21 +20887,17 @@ yynewstate: TblOptions: yyS[yypt-0].item.([]*ast.TableOption), } } - case 2399: + case 2400: { parser.yyVAL.item = []*ast.SequenceOption{} } - case 2401: - { - parser.yyVAL.item = []*ast.SequenceOption{yyS[yypt-0].item.(*ast.SequenceOption)} - } case 2402: { - parser.yyVAL.item = append(yyS[yypt-1].item.([]*ast.SequenceOption), yyS[yypt-0].item.(*ast.SequenceOption)) + parser.yyVAL.item = []*ast.SequenceOption{yyS[yypt-0].item.(*ast.SequenceOption)} } case 2403: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceOptionIncrementBy, IntValue: yyS[yypt-0].item.(int64)} + parser.yyVAL.item = append(yyS[yypt-1].item.([]*ast.SequenceOption), yyS[yypt-0].item.(*ast.SequenceOption)) } case 2404: { @@ -20892,7 +20905,7 @@ yynewstate: } case 2405: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceStartWith, IntValue: yyS[yypt-0].item.(int64)} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceOptionIncrementBy, IntValue: yyS[yypt-0].item.(int64)} } case 2406: { @@ -20900,11 +20913,11 @@ yynewstate: } case 2407: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceMinValue, IntValue: yyS[yypt-0].item.(int64)} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceStartWith, IntValue: yyS[yypt-0].item.(int64)} } case 2408: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoMinValue} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceMinValue, IntValue: yyS[yypt-0].item.(int64)} } case 2409: { @@ -20912,11 +20925,11 @@ yynewstate: } case 2410: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceMaxValue, IntValue: yyS[yypt-0].item.(int64)} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoMinValue} } case 2411: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoMaxValue} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceMaxValue, IntValue: yyS[yypt-0].item.(int64)} } case 2412: { @@ -20924,11 +20937,11 @@ yynewstate: } case 2413: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceCache, IntValue: yyS[yypt-0].item.(int64)} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoMaxValue} } case 2414: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoCache} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceCache, IntValue: yyS[yypt-0].item.(int64)} } case 2415: { @@ -20936,21 +20949,25 @@ yynewstate: } case 2416: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceCycle} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoCache} } case 2417: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoCycle} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceCycle} } case 2418: { parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoCycle} } - case 2420: + case 2419: { - parser.yyVAL.item = yyS[yypt-0].item + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoCycle} } case 2421: + { + parser.yyVAL.item = yyS[yypt-0].item + } + case 2422: { unsigned_num := getUint64FromNUM(yyS[yypt-0].item) if unsigned_num > 9223372036854775808 { @@ -20963,14 +20980,14 @@ yynewstate: parser.yyVAL.item = -int64(unsigned_num) } } - case 2422: + case 2423: { parser.yyVAL.statement = &ast.DropSequenceStmt{ IfExists: yyS[yypt-1].item.(bool), Sequences: yyS[yypt-0].item.([]*ast.TableName), } } - case 2423: + case 2424: { parser.yyVAL.statement = &ast.AlterSequenceStmt{ IfExists: yyS[yypt-2].item.(bool), @@ -20978,27 +20995,27 @@ yynewstate: SeqOptions: yyS[yypt-0].item.([]*ast.SequenceOption), } } - case 2424: + case 2425: { parser.yyVAL.item = []*ast.SequenceOption{yyS[yypt-0].item.(*ast.SequenceOption)} } - case 2425: + case 2426: { parser.yyVAL.item = append(yyS[yypt-1].item.([]*ast.SequenceOption), yyS[yypt-0].item.(*ast.SequenceOption)) } - case 2427: + case 2428: { parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceRestart} } - case 2428: + case 2429: { parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceRestartWith, IntValue: yyS[yypt-0].item.(int64)} } - case 2429: + case 2430: { parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceRestartWith, IntValue: yyS[yypt-0].item.(int64)} } - case 2430: + case 2431: { x := &ast.IndexAdviseStmt{ Path: yyS[yypt-3].ident, @@ -21015,42 +21032,42 @@ yynewstate: } parser.yyVAL.statement = x } - case 2431: + case 2432: { parser.yyVAL.item = uint64(ast.UnspecifiedSize) } - case 2432: + case 2433: { parser.yyVAL.item = getUint64FromNUM(yyS[yypt-0].item) } - case 2433: + case 2434: { parser.yyVAL.item = nil } - case 2434: + case 2435: { parser.yyVAL.item = &ast.MaxIndexNumClause{ PerTable: yyS[yypt-1].item.(uint64), PerDB: yyS[yypt-0].item.(uint64), } } - case 2435: + case 2436: { parser.yyVAL.item = uint64(ast.UnspecifiedSize) } - case 2436: + case 2437: { parser.yyVAL.item = getUint64FromNUM(yyS[yypt-0].item) } - case 2437: + case 2438: { parser.yyVAL.item = uint64(ast.UnspecifiedSize) } - case 2438: + case 2439: { parser.yyVAL.item = getUint64FromNUM(yyS[yypt-0].item) } - case 2439: + case 2440: { // Parse it but will ignore it switch yyS[yypt-0].ident { @@ -21065,19 +21082,19 @@ yynewstate: } parser.yyVAL.ident = yyS[yypt-0].ident } - case 2440: + case 2441: { parser.yyVAL.item = append([]*ast.RowExpr{}, yyS[yypt-0].item.(*ast.RowExpr)) } - case 2441: + case 2442: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*ast.RowExpr), yyS[yypt-0].item.(*ast.RowExpr)) } - case 2442: + case 2443: { parser.yyVAL.item = &ast.RowExpr{Values: yyS[yypt-0].item.([]ast.ExprNode)} } - case 2443: + case 2444: { x := &ast.PlanReplayerStmt{ Stmt: yyS[yypt-0].statement, @@ -21093,7 +21110,7 @@ yynewstate: parser.yyVAL.statement = x } - case 2444: + case 2445: { x := &ast.PlanReplayerStmt{ Stmt: yyS[yypt-0].statement, @@ -21109,7 +21126,7 @@ yynewstate: parser.yyVAL.statement = x } - case 2445: + case 2446: { x := &ast.PlanReplayerStmt{ Stmt: nil, @@ -21129,7 +21146,7 @@ yynewstate: parser.yyVAL.statement = x } - case 2446: + case 2447: { x := &ast.PlanReplayerStmt{ Stmt: nil, @@ -21149,7 +21166,7 @@ yynewstate: parser.yyVAL.statement = x } - case 2447: + case 2448: { x := &ast.PlanReplayerStmt{ Stmt: nil, diff --git a/parser/parser.y b/parser/parser.y index 6d6507b5ad604..ca3b415a3fada 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -4521,8 +4521,9 @@ TraceStmt: "TRACE" TraceableStmt { $$ = &ast.TraceStmt{ - Stmt: $2, - Format: "row", + Stmt: $2, + Format: "row", + TracePlan: false, } startOffset := parser.startOffset(&yyS[yypt]) $2.SetText(string(parser.src[startOffset:])) @@ -4530,12 +4531,22 @@ TraceStmt: | "TRACE" "FORMAT" "=" stringLit TraceableStmt { $$ = &ast.TraceStmt{ - Stmt: $5, - Format: $4, + Stmt: $5, + Format: $4, + TracePlan: false, } startOffset := parser.startOffset(&yyS[yypt]) $5.SetText(string(parser.src[startOffset:])) } +| "TRACE" "PLAN" TraceableStmt + { + $$ = &ast.TraceStmt{ + Stmt: $3, + TracePlan: true, + } + startOffset := parser.startOffset(&yyS[yypt]) + $3.SetText(string(parser.src[startOffset:])) + } ExplainSym: "EXPLAIN" @@ -9690,7 +9701,7 @@ VariableAssignment: { $$ = &ast.VariableAssignment{Name: $2, Value: $4, IsSystem: true} } -| "LOCAL" VariableName EqOrAssignmentEq Expression +| "LOCAL" VariableName EqOrAssignmentEq SetExpr { $$ = &ast.VariableAssignment{Name: $2, Value: $4, IsSystem: true} } @@ -13299,12 +13310,13 @@ DropPolicyStmt: } CreatePolicyStmt: - "CREATE" "PLACEMENT" "POLICY" IfNotExists PolicyName PlacementOptionList + "CREATE" OrReplace "PLACEMENT" "POLICY" IfNotExists PolicyName PlacementOptionList { $$ = &ast.CreatePlacementPolicyStmt{ - IfNotExists: $4.(bool), - PolicyName: model.NewCIStr($5), - PlacementOptions: $6.([]*ast.PlacementOption), + OrReplace: $2.(bool), + IfNotExists: $5.(bool), + PolicyName: model.NewCIStr($6), + PlacementOptions: $7.([]*ast.PlacementOption), } } diff --git a/parser/parser_test.go b/parser/parser_test.go index acea251d3ac30..89e26b3edb557 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -1298,10 +1298,12 @@ func TestSetVariable(t *testing.T) { {"set xx.xx = 666", "xx.xx", false, true}, // Set session system variable xx.xx {"set session xx.xx = 666", "xx.xx", false, true}, + {"set local xx.xx = 666", "xx.xx", false, true}, {"set global xx.xx = 666", "xx.xx", true, true}, {"set @@xx.xx = 666", "xx.xx", false, true}, {"set @@session.xx.xx = 666", "xx.xx", false, true}, + {"set @@local.xx.xx = 666", "xx.xx", false, true}, {"set @@global.xx.xx = 666", "xx.xx", true, true}, // Set user defined variable xx.xx @@ -3403,6 +3405,7 @@ func TestDDL(t *testing.T) { {"create placement policy x primary_region='cn' regions='us' schedule='even'", true, "CREATE PLACEMENT POLICY `x` PRIMARY_REGION = 'cn' REGIONS = 'us' SCHEDULE = 'even'"}, {"create placement policy x primary_region='cn', leader_constraints='ww', leader_constraints='yy'", true, "CREATE PLACEMENT POLICY `x` PRIMARY_REGION = 'cn' LEADER_CONSTRAINTS = 'ww' LEADER_CONSTRAINTS = 'yy'"}, {"create placement policy if not exists x regions = 'us', follower_constraints='yy'", true, "CREATE PLACEMENT POLICY IF NOT EXISTS `x` REGIONS = 'us' FOLLOWER_CONSTRAINTS = 'yy'"}, + {"create or replace placement policy x regions='us'", true, "CREATE OR REPLACE PLACEMENT POLICY `x` REGIONS = 'us'"}, {"create placement policy x placement policy y", false, ""}, {"alter placement policy x primary_region='us'", true, "ALTER PLACEMENT POLICY `x` PRIMARY_REGION = 'us'"}, @@ -4763,6 +4766,7 @@ func TestTrace(t *testing.T) { {"trace select c1 from t1 union (select c2 from t2) limit 1, 1", true, "TRACE SELECT `c1` FROM `t1` UNION (SELECT `c2` FROM `t2`) LIMIT 1,1"}, {"trace format = 'row' select c1 from t1 union (select c2 from t2) limit 1, 1", true, "TRACE SELECT `c1` FROM `t1` UNION (SELECT `c2` FROM `t2`) LIMIT 1,1"}, {"trace format = 'json' update t set id = id + 1 order by id desc;", true, "TRACE FORMAT = 'json' UPDATE `t` SET `id`=`id`+1 ORDER BY `id` DESC"}, + {"trace plan select c1 from t1", true, "TRACE PLAN SELECT `c1` FROM `t1`"}, } RunTest(t, table, false) } diff --git a/parser/reserved_words_test.go b/parser/reserved_words_test.go index fe581a52167d1..28821c3def11e 100644 --- a/parser/reserved_words_test.go +++ b/parser/reserved_words_test.go @@ -11,7 +11,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -//+build reserved_words_test +//go:build reserved_words_test +// +build reserved_words_test // This file ensures that the set of reserved keywords is the same as that of // MySQL. To run: diff --git a/parser/test_driver/test_driver.go b/parser/test_driver/test_driver.go index 857cdeb320f4c..5a4ae2ae13681 100644 --- a/parser/test_driver/test_driver.go +++ b/parser/test_driver/test_driver.go @@ -11,7 +11,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -//+build !codes +//go:build !codes +// +build !codes package test_driver diff --git a/parser/test_driver/test_driver_datum.go b/parser/test_driver/test_driver_datum.go index 46bda5cce08d3..9caa494b2153d 100644 --- a/parser/test_driver/test_driver_datum.go +++ b/parser/test_driver/test_driver_datum.go @@ -11,7 +11,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -//+build !codes +//go:build !codes +// +build !codes package test_driver diff --git a/parser/test_driver/test_driver_helper.go b/parser/test_driver/test_driver_helper.go index 3ecd2653cf9b9..5ea4f5ca1dd8c 100644 --- a/parser/test_driver/test_driver_helper.go +++ b/parser/test_driver/test_driver_helper.go @@ -11,7 +11,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -//+build !codes +//go:build !codes +// +build !codes package test_driver diff --git a/parser/test_driver/test_driver_mydecimal.go b/parser/test_driver/test_driver_mydecimal.go index abd3f9403ebee..9632cf6db5134 100644 --- a/parser/test_driver/test_driver_mydecimal.go +++ b/parser/test_driver/test_driver_mydecimal.go @@ -11,7 +11,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -//+build !codes +//go:build !codes +// +build !codes package test_driver diff --git a/parser/types/etc.go b/parser/types/etc.go index adc17a4f52233..2fe3d113e8820 100644 --- a/parser/types/etc.go +++ b/parser/types/etc.go @@ -151,5 +151,13 @@ const ( wordSize = 4 // A word is 4 bytes int32. ) -// ErrInvalidDefault is returned when meet a invalid default value. -var ErrInvalidDefault = terror.ClassTypes.NewStd(mysql.ErrInvalidDefault) +var ( + // ErrInvalidDefault is returned when meet a invalid default value. + ErrInvalidDefault = terror.ClassTypes.NewStd(mysql.ErrInvalidDefault) + // ErrDataOutOfRange is returned when meet a value out of range. + ErrDataOutOfRange = terror.ClassTypes.NewStd(mysql.ErrDataOutOfRange) + // ErrTruncatedWrongValue is returned when meet a value bigger than 99999999999999999999999999999999999999999999999999999999999999999 during parsing. + ErrTruncatedWrongValue = terror.ClassTypes.NewStd(mysql.ErrTruncatedWrongValue) + // ErrIllegalValueForType is returned when strconv.ParseFloat meet strconv.ErrRange during parsing. + ErrIllegalValueForType = terror.ClassTypes.NewStd(mysql.ErrIllegalValueForType) +) diff --git a/parser/yy_parser.go b/parser/yy_parser.go index ba11258a8d040..df3f416fad2e7 100644 --- a/parser/yy_parser.go +++ b/parser/yy_parser.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/parser/types" ) var ( @@ -272,7 +273,12 @@ func toInt(l yyLexer, lval *yySymType, str string) int { func toDecimal(l yyLexer, lval *yySymType, str string) int { dec, err := ast.NewDecimal(str) if err != nil { - l.AppendError(l.Errorf("decimal literal: %v", err)) + if terror.ErrorEqual(err, types.ErrDataOutOfRange) { + l.AppendWarn(types.ErrTruncatedWrongValue.FastGenByArgs("DECIMAL", dec)) + dec, _ = ast.NewDecimal(mysql.DefaultDecimal) + } else { + l.AppendError(l.Errorf("decimal literal: %v", err)) + } } lval.item = dec return decLit @@ -281,6 +287,11 @@ func toDecimal(l yyLexer, lval *yySymType, str string) int { func toFloat(l yyLexer, lval *yySymType, str string) int { n, err := strconv.ParseFloat(str, 64) if err != nil { + e := err.(*strconv.NumError) + if e.Err == strconv.ErrRange { + l.AppendError(types.ErrIllegalValueForType.GenWithStackByArgs("double", str)) + return int(unicode.ReplacementChar) + } l.AppendError(l.Errorf("float literal: %v", err)) return int(unicode.ReplacementChar) } @@ -385,7 +396,7 @@ var ( func resetParams(p *Parser) { p.charset = mysql.DefaultCharset p.collation = mysql.DefaultCollationName - p.lexer.encoding = charset.Encoding{} + p.lexer.encoding = charset.UTF8Encoding } // ParseParam represents the parameter of parsing. @@ -425,6 +436,6 @@ type CharsetClient string // ApplyOn implements ParseParam interface. func (c CharsetClient) ApplyOn(p *Parser) error { - p.lexer.encoding = *charset.NewEncoding(string(c)) + p.lexer.encoding = charset.NewEncoding(string(c)) return nil } diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index d69f0fa05d761..13029e49847f4 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -381,11 +381,26 @@ func (e *Execute) setFoundInPlanCache(sctx sessionctx.Context, opt bool) error { } func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, preparedStmt *CachedPrepareStmt) error { + var cacheKey kvcache.Key sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx prepared := preparedStmt.PreparedAst + if prepared.UseCache { + // disable the cache if cache table in prepared statement + for _, vInfo := range preparedStmt.VisitInfos { + tbl, err := is.TableByName(model.NewCIStr(vInfo.db), model.NewCIStr(vInfo.table)) + // if table does not exist, skip it, maybe it is a `create table` statement + if err != nil { + continue + } + if tbl.Meta().TableCacheStatusType == model.TableCacheStatusEnable { + prepared.UseCache = false + break + } + } + } stmtCtx.UseCache = prepared.UseCache - var cacheKey kvcache.Key + if prepared.UseCache { cacheKey = NewPSTMTPlanCacheKey(sctx.GetSessionVars(), e.ExecID, prepared.SchemaVersion) } @@ -606,7 +621,7 @@ func (e *Execute) rebuildRange(p Plan) error { } } if pkCol != nil { - ranges, err := ranger.BuildTableRange(x.AccessConditions, x.ctx.GetSessionVars().StmtCtx, pkCol.RetType) + ranges, err := ranger.BuildTableRange(x.AccessConditions, x.ctx, pkCol.RetType) if err != nil { return err } @@ -656,7 +671,7 @@ func (e *Execute) rebuildRange(p Plan) error { } } if pkCol != nil { - ranges, err := ranger.BuildTableRange(x.AccessConditions, x.ctx.GetSessionVars().StmtCtx, pkCol.RetType) + ranges, err := ranger.BuildTableRange(x.AccessConditions, x.ctx, pkCol.RetType) if err != nil { return err } @@ -759,7 +774,7 @@ func (e *Execute) buildRangeForTableScan(sctx sessionctx.Context, ts *PhysicalTa } } if pkCol != nil { - ts.Ranges, err = ranger.BuildTableRange(ts.AccessCondition, sctx.GetSessionVars().StmtCtx, pkCol.RetType) + ts.Ranges, err = ranger.BuildTableRange(ts.AccessCondition, sctx, pkCol.RetType) if err != nil { return err } @@ -1283,6 +1298,8 @@ func (e *Explain) explainPlanInRowFormat(p Plan, taskType, driverSide, indent st } case *PhysicalCTE: e.ctes = append(e.ctes, x) + case *PhysicalShuffleReceiverStub: + err = e.explainPlanInRowFormat(x.DataSource, "root", "", childIndent, true) } return } @@ -1469,10 +1486,10 @@ func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p Plan) (bo switch v := p.(type) { case *PhysicalIndexReader: indexScan := v.IndexPlans[0].(*PhysicalIndexScan) - return indexScan.IsPointGetByUniqueKey(ctx.GetSessionVars().StmtCtx), nil + return indexScan.IsPointGetByUniqueKey(ctx), nil case *PhysicalTableReader: tableScan := v.TablePlans[0].(*PhysicalTableScan) - isPointRange := len(tableScan.Ranges) == 1 && tableScan.Ranges[0].IsPoint(ctx.GetSessionVars().StmtCtx) + isPointRange := len(tableScan.Ranges) == 1 && tableScan.Ranges[0].IsPoint(ctx) if !isPointRange { return false, nil } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index d09f5a00b76d5..fb6a02d34b484 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1206,7 +1206,7 @@ func (cwc *ColWithCmpFuncManager) BuildRangesByRow(ctx sessionctx.Context, row c } exprs = append(exprs, newExpr) // nozero } - ranges, err := ranger.BuildColumnRange(exprs, ctx.GetSessionVars().StmtCtx, cwc.TargetCol.RetType, cwc.colLength) + ranges, err := ranger.BuildColumnRange(exprs, ctx, cwc.TargetCol.RetType, cwc.colLength) if err != nil { return nil, err } @@ -1261,18 +1261,18 @@ func (ijHelper *indexJoinBuildHelper) resetContextForIndex(innerKeys []*expressi // usefulEqOrInFilters is the continuous eq/in conditions on current unused index columns. // uselessFilters is the conditions which cannot be used for building ranges. // remainingRangeCandidates is the other conditions for future use. -func (ijHelper *indexJoinBuildHelper) findUsefulEqAndInFilters(innerPlan *DataSource) (usefulEqOrInFilters, uselessFilters, remainingRangeCandidates []expression.Expression) { +func (ijHelper *indexJoinBuildHelper) findUsefulEqAndInFilters(innerPlan *DataSource) (usefulEqOrInFilters, uselessFilters, remainingRangeCandidates []expression.Expression, emptyRange bool) { uselessFilters = make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) var remainedEqOrIn []expression.Expression // Extract the eq/in functions of possible join key. // you can see the comment of ExtractEqAndInCondition to get the meaning of the second return value. - usefulEqOrInFilters, remainedEqOrIn, remainingRangeCandidates, _, _ = ranger.ExtractEqAndInCondition( + usefulEqOrInFilters, remainedEqOrIn, remainingRangeCandidates, _, emptyRange = ranger.ExtractEqAndInCondition( innerPlan.ctx, innerPlan.pushedDownConds, ijHelper.curNotUsedIndexCols, ijHelper.curNotUsedColLens, ) uselessFilters = append(uselessFilters, remainedEqOrIn...) - return usefulEqOrInFilters, uselessFilters, remainingRangeCandidates + return usefulEqOrInFilters, uselessFilters, remainingRangeCandidates, emptyRange } // buildLastColManager analyze the `OtherConditions` of join to see whether there're some filters can be used in manager. @@ -1394,7 +1394,10 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath accesses := make([]expression.Expression, 0, len(path.IdxCols)) relatedExprs := make([]expression.Expression, 0, len(path.IdxCols)) // all expressions related to the chosen range ijHelper.resetContextForIndex(innerJoinKeys, path.IdxCols, path.IdxColLens, outerJoinKeys) - notKeyEqAndIn, remained, rangeFilterCandidates := ijHelper.findUsefulEqAndInFilters(innerPlan) + notKeyEqAndIn, remained, rangeFilterCandidates, emptyRange := ijHelper.findUsefulEqAndInFilters(innerPlan) + if emptyRange { + return true, nil + } var remainedEqAndIn []expression.Expression notKeyEqAndIn, remainedEqAndIn = ijHelper.removeUselessEqAndInFunc(path.IdxCols, notKeyEqAndIn, outerJoinKeys) matchedKeyCnt := len(ijHelper.curPossibleUsedKeys) @@ -1450,7 +1453,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath var ranges, nextColRange []*ranger.Range var err error if len(colAccesses) > 0 { - nextColRange, err = ranger.BuildColumnRange(colAccesses, ijHelper.join.ctx.GetSessionVars().StmtCtx, lastPossibleCol.RetType, path.IdxColLens[lastColPos]) + nextColRange, err = ranger.BuildColumnRange(colAccesses, ijHelper.join.ctx, lastPossibleCol.RetType, path.IdxColLens[lastColPos]) if err != nil { return false, err } @@ -1559,13 +1562,16 @@ func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAn continue } exprs := []expression.Expression{eqAndInFuncs[j]} - oneColumnRan, err := ranger.BuildColumnRange(exprs, sc, ijHelper.curNotUsedIndexCols[j].RetType, ijHelper.curNotUsedColLens[j]) + oneColumnRan, err := ranger.BuildColumnRange(exprs, ijHelper.join.ctx, ijHelper.curNotUsedIndexCols[j].RetType, ijHelper.curNotUsedColLens[j]) if err != nil { return nil, false, err } if len(oneColumnRan) == 0 { return nil, true, nil } + if sc.MemTracker != nil { + sc.MemTracker.Consume(2 * types.EstimatedMemUsage(oneColumnRan[0].LowVal, len(oneColumnRan))) + } for _, ran := range ranges { ran.LowVal[i] = oneColumnRan[0].LowVal[0] ran.HighVal[i] = oneColumnRan[0].HighVal[0] @@ -1579,6 +1585,9 @@ func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAn newRange.HighVal[i] = oneColumnRan[ranIdx].HighVal[0] newRanges = append(newRanges, newRange) } + if sc.MemTracker != nil && len(newRanges) != 0 { + sc.MemTracker.Consume(2 * types.EstimatedMemUsage(newRanges[0].LowVal, len(newRanges))) + } ranges = append(ranges, newRanges...) } j++ diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index b2c270b923431..5bea6442b9101 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -1865,13 +1865,13 @@ func findFieldNameFromNaturalUsingJoin(p LogicalPlan, v *ast.ColumnName) (col *e case *LogicalLimit, *LogicalSelection, *LogicalTopN, *LogicalSort, *LogicalMaxOneRow: return findFieldNameFromNaturalUsingJoin(p.Children()[0], v) case *LogicalJoin: - if x.redundantSchema != nil { - idx, err := expression.FindFieldName(x.redundantNames, v) + if x.fullSchema != nil { + idx, err := expression.FindFieldName(x.fullNames, v) if err != nil { return nil, nil, err } if idx >= 0 { - return x.redundantSchema.Columns[idx], x.redundantNames[idx], nil + return x.fullSchema.Columns[idx], x.fullNames[idx], nil } } } @@ -2059,7 +2059,8 @@ func decodeRecordKey(key []byte, tableID int64, tbl table.Table, loc *time.Locat ret := make(map[string]interface{}) ret["table_id"] = tableID handleRet := make(map[string]interface{}) - for colID, dt := range datumMap { + for colID := range datumMap { + dt := datumMap[colID] dtStr, err := datumToJSONObject(&dt) if err != nil { return "", errors.Trace(err) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 57965d2fb91e7..3e13a6bb6aea1 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -812,10 +812,12 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter p: dual, }, cntPlan, nil } - canConvertPointGet := len(path.Ranges) > 0 && path.StoreType == kv.TiKV && ds.isPointGetConvertableSchema() && - // to avoid the over-optimized risk, do not generate PointGet for plan cache, for example, - // `pk>=$a and pk<=$b` can be optimized to a PointGet when `$a==$b`, but it can cause wrong results when `$a!=$b`. - !ds.ctx.GetSessionVars().StmtCtx.UseCache + canConvertPointGet := len(path.Ranges) > 0 && path.StoreType == kv.TiKV && ds.isPointGetConvertableSchema() + + if canConvertPointGet && expression.MaybeOverOptimized4PlanCache(ds.ctx, path.AccessConds) { + canConvertPointGet = ds.canConvertToPointGetForPlanCache(path) + } + if canConvertPointGet && !path.IsIntHandlePath { // We simply do not build [batch] point get for prefix indexes. This can be optimized. canConvertPointGet = path.Index.Unique && !path.Index.HasPrefixIndex() @@ -858,7 +860,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter if canConvertPointGet { allRangeIsPoint := true for _, ran := range path.Ranges { - if !ran.IsPoint(ds.ctx.GetSessionVars().StmtCtx) { + if !ran.IsPoint(ds.ctx) { allRangeIsPoint = false break } @@ -934,6 +936,27 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter return } +func (ds *DataSource) canConvertToPointGetForPlanCache(path *util.AccessPath) bool { + // PointGet might contain some over-optimized assumptions, like `a>=1 and a<=1` --> `a=1`, but + // these assumptions may be broken after parameters change. + // So for safety, we narrow down the scope and just generate PointGet in some particular and simple scenarios. + + // scenario 1: each column corresponds to a single EQ, `a=1 and b=2 and c=3` --> `[1, 2, 3]` + if len(path.Ranges) > 0 && path.Ranges[0].Width() == len(path.AccessConds) { + for _, accessCond := range path.AccessConds { + f, ok := accessCond.(*expression.ScalarFunction) + if !ok { + return false + } + if f.FuncName.L != ast.EQ { + return false + } + } + return true + } + return false +} + func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, candidate *candidatePath) (task task, err error) { if prop.TaskTp != property.RootTaskType || !prop.IsEmpty() { return invalidTask, nil @@ -1558,7 +1581,7 @@ func (ds *DataSource) crossEstimateRowCount(path *util.AccessPath, conds []expre return 0, false, corr } sc := ds.ctx.GetSessionVars().StmtCtx - ranges, err := ranger.BuildColumnRange(accessConds, sc, col.RetType, types.UnspecifiedLength) + ranges, err := ranger.BuildColumnRange(accessConds, ds.ctx, col.RetType, types.UnspecifiedLength) if len(ranges) == 0 || err != nil { return 0, err == nil, corr } @@ -1824,8 +1847,8 @@ func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candida var partitionInfo *model.PartitionDefinition if ds.isPartition { if pi := ds.tableInfo.GetPartitionInfo(); pi != nil { - for _, def := range pi.Definitions { - def := def + for i := range pi.Definitions { + def := pi.Definitions[i] if def.ID == ds.physicalTableID { partitionInfo = &def break diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 5889d3725330f..8fb6c65858989 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -4311,6 +4311,18 @@ func (s *testIntegrationSuite) TestCreateViewWithWindowFunc(c *C) { rows.Check(testkit.Rows("1 1")) } +func (s *testIntegrationSuite) TestIssue29834(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists IDT_MC21814;") + tk.MustExec("CREATE TABLE `IDT_MC21814` (`COL1` year(4) DEFAULT NULL,`COL2` year(4) DEFAULT NULL,KEY `U_M_COL` (`COL1`,`COL2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;") + tk.MustExec("insert into IDT_MC21814 values(1901, 2119), (2155, 2000);") + tk.MustQuery("SELECT/*+ INL_JOIN(t1, t2), nth_plan(1) */ t2.* FROM IDT_MC21814 t1 LEFT JOIN IDT_MC21814 t2 ON t1.col1 = t2.col1 WHERE t2.col2 BETWEEN 2593 AND 1971 AND t1.col1 IN (2155, 1901, 1967);").Check(testkit.Rows()) + tk.MustQuery("SELECT/*+ INL_JOIN(t1, t2), nth_plan(2) */ t2.* FROM IDT_MC21814 t1 LEFT JOIN IDT_MC21814 t2 ON t1.col1 = t2.col1 WHERE t2.col2 BETWEEN 2593 AND 1971 AND t1.col1 IN (2155, 1901, 1967);").Check(testkit.Rows()) + // Only can generate one index join plan. Because the index join inner child can not be tableDual. + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 The parameter of nth_plan() is out of range.")) +} + func (s *testIntegrationSuite) TestIssue29221(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -4707,6 +4719,26 @@ func (s *testIntegrationSuite) TestIssue27797(c *C) { result.Check(testkit.Rows("")) } +func (s *testIntegrationSuite) TestIssue27949(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t27949") + tk.MustExec("create table t27949 (a int, b int, key(b))") + tk.MustQuery("explain format = 'brief' select * from t27949 where b=1").Check(testkit.Rows("IndexLookUp 10.00 root ", + "├─IndexRangeScan(Build) 10.00 cop[tikv] table:t27949, index:b(b) range:[1,1], keep order:false, stats:pseudo", + "└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t27949 keep order:false, stats:pseudo")) + tk.MustExec("create global binding for select * from t27949 where b=1 using select * from t27949 ignore index(b) where b=1") + tk.MustQuery("explain format = 'brief' select * from t27949 where b=1").Check(testkit.Rows("TableReader 10.00 root data:Selection", + "└─Selection 10.00 cop[tikv] eq(test.t27949.b, 1)", + " └─TableFullScan 10000.00 cop[tikv] table:t27949 keep order:false, stats:pseudo")) + tk.MustExec("set @@sql_select_limit=100") + tk.MustQuery("explain format = 'brief' select * from t27949 where b=1").Check(testkit.Rows("Limit 10.00 root offset:0, count:100", + "└─TableReader 10.00 root data:Limit", + " └─Limit 10.00 cop[tikv] offset:0, count:100", + " └─Selection 10.00 cop[tikv] eq(test.t27949.b, 1)", + " └─TableFullScan 10000.00 cop[tikv] table:t27949 keep order:false, stats:pseudo")) +} + func (s *testIntegrationSuite) TestIssue28154(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index e1c9c11287cb2..02451a4c4cb65 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -277,8 +277,8 @@ func (b *PlanBuilder) buildAggregation(ctx context.Context, p LogicalPlan, aggFu schema4Agg.Append(newCol) names = append(names, p.OutputNames()[i]) } - if join, isJoin := p.(*LogicalJoin); isJoin && join.redundantSchema != nil { - for i, col := range join.redundantSchema.Columns { + if join, isJoin := p.(*LogicalJoin); isJoin && join.fullSchema != nil { + for i, col := range join.fullSchema.Columns { if p.Schema().Contains(col) { continue } @@ -290,7 +290,7 @@ func (b *PlanBuilder) buildAggregation(ctx context.Context, p LogicalPlan, aggFu newCol, _ := col.Clone().(*expression.Column) newCol.RetType = newFunc.RetTp schema4Agg.Append(newCol) - names = append(names, join.redundantNames[i]) + names = append(names, join.fullNames[i]) } } hasGroupBy := len(gbyItems) > 0 @@ -721,25 +721,50 @@ func (b *PlanBuilder) buildJoin(ctx context.Context, joinNode *ast.Join) (Logica joinPlan.JoinType = InnerJoin } - // Merge sub join's redundantSchema into this join plan. When handle query like - // select t2.a from (t1 join t2 using (a)) join t3 using (a); - // we can simply search in the top level join plan to find redundant column. + // Merge sub-plan's fullSchema into this join plan. + // Please read the comment of LogicalJoin.fullSchema for the details. var ( - lRedundantSchema, rRedundantSchema *expression.Schema - lRedundantNames, rRedundantNames types.NameSlice + lFullSchema, rFullSchema *expression.Schema + lFullNames, rFullNames types.NameSlice ) - if left, ok := leftPlan.(*LogicalJoin); ok && left.redundantSchema != nil { - lRedundantSchema = left.redundantSchema - lRedundantNames = left.redundantNames + if left, ok := leftPlan.(*LogicalJoin); ok && left.fullSchema != nil { + lFullSchema = left.fullSchema + lFullNames = left.fullNames + } else { + lFullSchema = leftPlan.Schema() + lFullNames = leftPlan.OutputNames() + } + if right, ok := rightPlan.(*LogicalJoin); ok && right.fullSchema != nil { + rFullSchema = right.fullSchema + rFullNames = right.fullNames + } else { + rFullSchema = rightPlan.Schema() + rFullNames = rightPlan.OutputNames() } - if right, ok := rightPlan.(*LogicalJoin); ok && right.redundantSchema != nil { - rRedundantSchema = right.redundantSchema - rRedundantNames = right.redundantNames + if joinNode.Tp == ast.RightJoin { + // Make sure lFullSchema means outer full schema and rFullSchema means inner full schema. + lFullSchema, rFullSchema = rFullSchema, lFullSchema + lFullNames, rFullNames = rFullNames, lFullNames + } + joinPlan.fullSchema = expression.MergeSchema(lFullSchema, rFullSchema) + + // Clear NotNull flag for the inner side schema if it's an outer join. + if joinNode.Tp == ast.LeftJoin || joinNode.Tp == ast.RightJoin { + resetNotNullFlag(joinPlan.fullSchema, lFullSchema.Len(), joinPlan.fullSchema.Len()) + } + + // Merge sub-plan's fullNames into this join plan, similar to the fullSchema logic above. + joinPlan.fullNames = make([]*types.FieldName, 0, len(lFullNames)+len(rFullNames)) + for _, lName := range lFullNames { + name := *lName + name.Redundant = true + joinPlan.fullNames = append(joinPlan.fullNames, &name) + } + for _, rName := range rFullNames { + name := *rName + name.Redundant = true + joinPlan.fullNames = append(joinPlan.fullNames, &name) } - joinPlan.redundantSchema = expression.MergeSchema(lRedundantSchema, rRedundantSchema) - joinPlan.redundantNames = make([]*types.FieldName, len(lRedundantNames)+len(rRedundantNames)) - copy(joinPlan.redundantNames, lRedundantNames) - copy(joinPlan.redundantNames[len(lRedundantNames):], rRedundantNames) // Set preferred join algorithm if some join hints is specified by user. joinPlan.setPreferredJoinType(b.TableHints()) @@ -942,21 +967,7 @@ func (b *PlanBuilder) coalesceCommonColumns(p *LogicalJoin, leftPlan, rightPlan p.SetSchema(expression.NewSchema(schemaCols...)) p.names = names - // We record the full `rightPlan.Schema` as `redundantSchema` in order to - // record the redundant column in `rightPlan` and the output columns order - // of the `rightPlan`. - // For SQL like `select t1.*, t2.* from t1 left join t2 using(a)`, we can - // retrieve the column order of `t2.*` from the `redundantSchema`. - p.redundantSchema = expression.MergeSchema(p.redundantSchema, expression.NewSchema(rightPlan.Schema().Clone().Columns...)) - p.redundantNames = p.redundantNames.Shallow() - for _, name := range rightPlan.OutputNames() { - cpyName := *name - cpyName.Redundant = true - p.redundantNames = append(p.redundantNames, &cpyName) - } - if joinTp == ast.RightJoin || joinTp == ast.LeftJoin { - resetNotNullFlag(p.redundantSchema, 0, p.redundantSchema.Len()) - } + p.OtherConditions = append(conds, p.OtherConditions...) return nil @@ -1209,9 +1220,9 @@ func findColFromNaturalUsingJoin(p LogicalPlan, col *expression.Column) (name *t case *LogicalLimit, *LogicalSelection, *LogicalTopN, *LogicalSort, *LogicalMaxOneRow: return findColFromNaturalUsingJoin(p.Children()[0], col) case *LogicalJoin: - if x.redundantSchema != nil { - idx := x.redundantSchema.ColumnIndex(col) - return x.redundantNames[idx] + if x.fullSchema != nil { + idx := x.fullSchema.ColumnIndex(col) + return x.fullNames[idx] } } return nil @@ -1997,9 +2008,9 @@ func (a *havingWindowAndOrderbyExprResolver) resolveFromPlan(v *ast.ColumnNameEx case *LogicalLimit, *LogicalSelection, *LogicalTopN, *LogicalSort, *LogicalMaxOneRow: return a.resolveFromPlan(v, p.Children()[0]) case *LogicalJoin: - if len(x.redundantNames) != 0 { - idx, err = expression.FindFieldName(x.redundantNames, v.Name) - schemaCols, outputNames = x.redundantSchema.Columns, x.redundantNames + if len(x.fullNames) != 0 { + idx, err = expression.FindFieldName(x.fullNames, v.Name) + schemaCols, outputNames = x.fullSchema.Columns, x.fullNames } } if err != nil || idx < 0 { @@ -3147,14 +3158,11 @@ func (b *PlanBuilder) unfoldWildStar(p LogicalPlan, selectFields []*ast.SelectFi return nil, ErrInvalidWildCard } list := unfoldWildStar(field, p.OutputNames(), p.Schema().Columns) - // For sql like `select t1.*, t2.* from t1 join t2 using(a)`, we should - // not coalesce the `t2.a` in the output result. Thus we need to unfold - // the wildstar from the underlying join.redundantSchema. - if isJoin && join.redundantSchema != nil && field.WildCard.Table.L != "" { - redundantList := unfoldWildStar(field, join.redundantNames, join.redundantSchema.Columns) - if len(redundantList) > len(list) { - list = redundantList - } + // For sql like `select t1.*, t2.* from t1 join t2 using(a)` or `select t1.*, t2.* from t1 natual join t2`, + // the schema of the Join doesn't contain enough columns because the join keys are coalesced in this schema. + // We should collect the columns from the fullSchema. + if isJoin && join.fullSchema != nil && field.WildCard.Table.L != "" { + list = unfoldWildStar(field, join.fullNames, join.fullSchema.Columns) } if len(list) == 0 { return nil, ErrBadTable.GenWithStackByArgs(field.WildCard.Table) @@ -3190,7 +3198,8 @@ func unfoldWildStar(field *ast.SelectField, outputName types.NameSlice, column [ return resultList } -func (b *PlanBuilder) addAliasName(ctx context.Context, selectFields []*ast.SelectField, p LogicalPlan) (resultList []*ast.SelectField, err error) { +func (b *PlanBuilder) addAliasName(ctx context.Context, selectStmt *ast.SelectStmt, p LogicalPlan) (resultList []*ast.SelectField, err error) { + selectFields := selectStmt.Fields.Fields projOutNames := make([]*types.FieldName, 0, len(selectFields)) for _, field := range selectFields { colNameField, isColumnNameExpr := field.Expr.(*ast.ColumnNameExpr) @@ -3217,13 +3226,49 @@ func (b *PlanBuilder) addAliasName(ctx context.Context, selectFields []*ast.Sele } } + // dedupMap is used for renaming a duplicated anonymous column + dedupMap := make(map[string]int) + anonymousFields := make([]bool, len(selectFields)) + for i, field := range selectFields { newField := *field if newField.AsName.L == "" { newField.AsName = projOutNames[i].ColName } + + if _, ok := field.Expr.(*ast.ColumnNameExpr); !ok && field.AsName.L == "" { + anonymousFields[i] = true + } else { + anonymousFields[i] = false + // dedupMap should be inited with all non-anonymous fields before renaming other duplicated anonymous fields + dedupMap[newField.AsName.L] = 0 + } + resultList = append(resultList, &newField) } + + // We should rename duplicated anonymous fields in the first SelectStmt of CreateViewStmt + // See: https://github.com/pingcap/tidb/issues/29326 + if selectStmt.AsViewSchema { + for i, field := range resultList { + if !anonymousFields[i] { + continue + } + + oldName := field.AsName + if dup, ok := dedupMap[field.AsName.L]; ok { + if dup == 0 { + field.AsName = model.NewCIStr(fmt.Sprintf("Name_exp_%s", field.AsName.O)) + } else { + field.AsName = model.NewCIStr(fmt.Sprintf("Name_exp_%d_%s", dup, field.AsName.O)) + } + dedupMap[oldName.L] = dup + 1 + } else { + dedupMap[oldName.L] = 0 + } + } + } + return resultList, nil } @@ -3517,7 +3562,7 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L } if b.capFlag&canExpandAST != 0 { // To be compabitle with MySQL, we add alias name for each select field when creating view. - sel.Fields.Fields, err = b.addAliasName(ctx, sel.Fields.Fields, p) + sel.Fields.Fields, err = b.addAliasName(ctx, sel, p) if err != nil { return nil, err } @@ -4158,18 +4203,18 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as us.SetChildren(ds) result = us } else { - go func() { - defer func() { - if r := recover(); r != nil { - } - }() - if !b.inUpdateStmt && !b.inDeleteStmt && !b.ctx.GetSessionVars().StmtCtx.InExplainStmt { + if !b.inUpdateStmt && !b.inDeleteStmt && !sessionVars.StmtCtx.InExplainStmt { + go func() { + defer func() { + if r := recover(); r != nil { + } + }() err := cachedTable.UpdateLockForRead(b.ctx.GetStore(), txn.StartTS()) if err != nil { log.Warn("Update Lock Info Error") } - } - }() + }() + } } } diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 2258e2b4de31d..3d5bbeeaa9216 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -147,10 +147,22 @@ type LogicalJoin struct { // Currently, only `aggregation push down` phase will set this. DefaultValues []types.Datum - // redundantSchema contains columns which are eliminated in join. - // For select * from a join b using (c); a.c will in output schema, and b.c will only in redundantSchema. - redundantSchema *expression.Schema - redundantNames types.NameSlice + // fullSchema contains all the columns that the Join can output. It's ordered as [outer schema..., inner schema...]. + // This is useful for natural joins and "using" joins. In these cases, the join key columns from the + // inner side (or the right side when it's an inner join) will not be in the schema of Join. + // But upper operators should be able to find those "redundant" columns, and the user also can specifically select + // those columns, so we put the "redundant" columns here to make them be able to be found. + // + // For example: + // create table t1(a int, b int); create table t2(a int, b int); + // select * from t1 join t2 using (b); + // schema of the Join will be [t1.b, t1.a, t2.a]; fullSchema will be [t1.a, t1.b, t2.a, t2.b]. + // + // We record all columns and keep them ordered is for correctly handling SQLs like + // select t1.*, t2.* from t1 join t2 using (b); + // (*PlanBuilder).unfoldWildStar() handles the schema for such case. + fullSchema *expression.Schema + fullNames types.NameSlice // equalCondOutCnt indicates the estimated count of joined rows after evaluating `EqualConditions`. equalCondOutCnt float64 @@ -832,7 +844,7 @@ func (ds *DataSource) deriveTablePathStats(path *util.AccessPath, conds []expres path.CountAfterAccess = 1 return nil } - path.Ranges, err = ranger.BuildTableRange(path.AccessConds, sc, pkCol.RetType) + path.Ranges, err = ranger.BuildTableRange(path.AccessConds, ds.ctx, pkCol.RetType) if err != nil { return err } diff --git a/planner/core/memtable_predicate_extractor.go b/planner/core/memtable_predicate_extractor.go index 7fd2e787dae11..923d025ac7d13 100644 --- a/planner/core/memtable_predicate_extractor.go +++ b/planner/core/memtable_predicate_extractor.go @@ -541,6 +541,24 @@ func (helper extractHelper) convertToTime(t int64) time.Time { return time.Unix(0, t) } +func (helper extractHelper) convertToBoolSlice(uint64Slice []uint64) []bool { + if len(uint64Slice) == 0 { + return []bool{false, true} + } + var res []bool + // use to keep res unique + b := make(map[bool]struct{}, 2) + for _, l := range uint64Slice { + tmpBool := (l == 1) + _, ok := b[tmpBool] + if !ok { + b[tmpBool] = struct{}{} + res = append(res, tmpBool) + } + } + return res +} + // ClusterTableExtractor is used to extract some predicates of cluster table. type ClusterTableExtractor struct { extractHelper @@ -697,6 +715,13 @@ func (e *ClusterLogTableExtractor) explainInfo(p *PhysicalMemTable) string { return s } +const ( + // HotRegionTypeRead hot read region. + HotRegionTypeRead = "read" + // HotRegionTypeWrite hot write region. + HotRegionTypeWrite = "write" +) + // HotRegionsHistoryTableExtractor is used to extract some predicates of `tidb_hot_regions_history` type HotRegionsHistoryTableExtractor struct { extractHelper @@ -723,8 +748,8 @@ type HotRegionsHistoryTableExtractor struct { // e.g: // 1. SELECT * FROM tidb_hot_regions_history WHERE is_learner=1 // 2. SELECT * FROM tidb_hot_regions_history WHERE is_learner in (0,1) -> request all - IsLearners []uint64 - IsLeaders []uint64 + IsLearners []bool + IsLeaders []bool // HotRegionTypes represents all hot region types we should filter in PD to reduce network IO. // e.g: @@ -741,7 +766,7 @@ func (e *HotRegionsHistoryTableExtractor) Extract( names []*types.FieldName, predicates []expression.Expression, ) []expression.Expression { - // Extract the `region_id/store_id/peer_id` columns. + // Extract the `region_id/store_id/peer_id` columns remained, regionIDSkipRequest, regionIDs := e.extractCol(schema, names, predicates, "region_id", false) remained, storeIDSkipRequest, storeIDs := e.extractCol(schema, names, remained, "store_id", false) remained, peerIDSkipRequest, peerIDs := e.extractCol(schema, names, remained, "peer_id", false) @@ -751,25 +776,34 @@ func (e *HotRegionsHistoryTableExtractor) Extract( return nil } - // Extract the is_learner/is_leader columns. + // Extract the is_learner/is_leader columns remained, isLearnerSkipRequest, isLearners := e.extractCol(schema, names, remained, "is_learner", false) remained, isLeaderSkipRequest, isLeaders := e.extractCol(schema, names, remained, "is_leader", false) - e.IsLearners, e.IsLeaders = e.parseUint64(isLearners), e.parseUint64(isLeaders) + isLearnersUint64, isLeadersUint64 := e.parseUint64(isLearners), e.parseUint64(isLeaders) e.SkipRequest = isLearnerSkipRequest || isLeaderSkipRequest if e.SkipRequest { return nil } + // uint64 slice to unique bool slice + e.IsLearners = e.convertToBoolSlice(isLearnersUint64) + e.IsLeaders = e.convertToBoolSlice(isLeadersUint64) - // Extract the `type` column. + // Extract the `type` column remained, typeSkipRequest, types := e.extractCol(schema, names, remained, "type", false) e.HotRegionTypes = types e.SkipRequest = typeSkipRequest if e.SkipRequest { return nil } + // Divide read-write into two requests because of time range overlap, + // PD use [type,time] as key of hot regions. + if e.HotRegionTypes.Count() == 0 { + e.HotRegionTypes.Insert(HotRegionTypeRead) + e.HotRegionTypes.Insert(HotRegionTypeWrite) + } remained, startTime, endTime := e.extractTimeRange(ctx, schema, names, remained, "update_time", ctx.GetSessionVars().StmtCtx.TimeZone) - // The time unit for search hot regions is millisecond. + // The time unit for search hot regions is millisecond startTime = startTime / int64(time.Millisecond) endTime = endTime / int64(time.Millisecond) e.StartTime = startTime @@ -808,10 +842,10 @@ func (e *HotRegionsHistoryTableExtractor) explainInfo(p *PhysicalMemTable) strin r.WriteString(fmt.Sprintf("peer_ids:[%s], ", extractStringFromUint64Slice(e.PeerIDs))) } if len(e.IsLearners) > 0 { - r.WriteString(fmt.Sprintf("roles:[%s], ", extractStringFromUint64Slice(e.IsLearners))) + r.WriteString(fmt.Sprintf("learner_roles:[%s], ", extractStringFromBoolSlice(e.IsLearners))) } if len(e.IsLeaders) > 0 { - r.WriteString(fmt.Sprintf("roles:[%s], ", extractStringFromUint64Slice(e.IsLeaders))) + r.WriteString(fmt.Sprintf("leader_roles:[%s], ", extractStringFromBoolSlice(e.IsLeaders))) } if len(e.HotRegionTypes) > 0 { r.WriteString(fmt.Sprintf("hot_region_types:[%s], ", extractStringFromStringSet(e.HotRegionTypes))) diff --git a/planner/core/memtable_predicate_extractor_test.go b/planner/core/memtable_predicate_extractor_test.go index 15f468d33a5c3..311b9ae4a8838 100644 --- a/planner/core/memtable_predicate_extractor_test.go +++ b/planner/core/memtable_predicate_extractor_test.go @@ -1058,7 +1058,7 @@ func (s *extractorSuite) TestTiDBHotRegionsHistoryTableExtractor(c *C) { skipRequest bool startTime, endTime int64 regionIDs, storeIDs, peerIDs []uint64 - isLearners, isLeaders []uint64 + isLearners, isLeaders []bool hotRegionTypes set.StringSet }{ // Test full data, it will not call Extract() and executor(retriver) will panic and remind user to add conditions to save network IO. @@ -1071,114 +1071,180 @@ func (s *extractorSuite) TestTiDBHotRegionsHistoryTableExtractor(c *C) { sql: "select * from information_schema.tidb_hot_regions_history where update_time='2019-10-10 10::10'", }, { - sql: "select * from information_schema.tidb_hot_regions_history where update_time='2019-10-10 10:10:10'", - startTime: timestamp(c, "2019-10-10 10:10:10"), - endTime: timestamp(c, "2019-10-10 10:10:10"), + sql: "select * from information_schema.tidb_hot_regions_history where update_time='2019-10-10 10:10:10'", + startTime: timestamp(c, "2019-10-10 10:10:10"), + endTime: timestamp(c, "2019-10-10 10:10:10"), + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where update_time>='2019-10-10 10:10:10' and update_time<='2019-10-11 10:10:10'", - startTime: timestamp(c, "2019-10-10 10:10:10"), - endTime: timestamp(c, "2019-10-11 10:10:10"), + sql: "select * from information_schema.tidb_hot_regions_history where update_time>='2019-10-10 10:10:10' and update_time<='2019-10-11 10:10:10'", + startTime: timestamp(c, "2019-10-10 10:10:10"), + endTime: timestamp(c, "2019-10-11 10:10:10"), + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where update_time>'2019-10-10 10:10:10' and update_time<'2019-10-11 10:10:10'", - startTime: timestamp(c, "2019-10-10 10:10:10") + 1, - endTime: timestamp(c, "2019-10-11 10:10:10") - 1, + sql: "select * from information_schema.tidb_hot_regions_history where update_time>'2019-10-10 10:10:10' and update_time<'2019-10-11 10:10:10'", + startTime: timestamp(c, "2019-10-10 10:10:10") + 1, + endTime: timestamp(c, "2019-10-11 10:10:10") - 1, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where update_time>='2019-10-10 10:10:10' and update_time<'2019-10-11 10:10:10'", - startTime: timestamp(c, "2019-10-10 10:10:10"), - endTime: timestamp(c, "2019-10-11 10:10:10") - 1, + sql: "select * from information_schema.tidb_hot_regions_history where update_time>='2019-10-10 10:10:10' and update_time<'2019-10-11 10:10:10'", + startTime: timestamp(c, "2019-10-10 10:10:10"), + endTime: timestamp(c, "2019-10-11 10:10:10") - 1, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where update_time>='2019-10-12 10:10:10' and update_time<'2019-10-11 10:10:10'", - startTime: timestamp(c, "2019-10-12 10:10:10"), - endTime: timestamp(c, "2019-10-11 10:10:10") - 1, - skipRequest: true, + sql: "select * from information_schema.tidb_hot_regions_history where update_time>='2019-10-12 10:10:10' and update_time<'2019-10-11 10:10:10'", + startTime: timestamp(c, "2019-10-12 10:10:10"), + endTime: timestamp(c, "2019-10-11 10:10:10") - 1, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), + skipRequest: true, }, { - sql: "select * from information_schema.tidb_hot_regions_history where update_time>='2019-10-10 10:10:10'", - startTime: timestamp(c, "2019-10-10 10:10:10"), + sql: "select * from information_schema.tidb_hot_regions_history where update_time>='2019-10-10 10:10:10'", + startTime: timestamp(c, "2019-10-10 10:10:10"), + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where update_time>='2019-10-10 10:10:10' and update_time>='2019-10-11 10:10:10' and update_time>='2019-10-12 10:10:10'", - startTime: timestamp(c, "2019-10-12 10:10:10"), + sql: "select * from information_schema.tidb_hot_regions_history where update_time>='2019-10-10 10:10:10' and update_time>='2019-10-11 10:10:10' and update_time>='2019-10-12 10:10:10'", + startTime: timestamp(c, "2019-10-12 10:10:10"), + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where update_time>='2019-10-10 10:10:10' and update_time>='2019-10-11 10:10:10' and update_time>='2019-10-12 10:10:10' and update_time='2019-10-13 10:10:10'", - startTime: timestamp(c, "2019-10-13 10:10:10"), - endTime: timestamp(c, "2019-10-13 10:10:10"), + sql: "select * from information_schema.tidb_hot_regions_history where update_time>='2019-10-10 10:10:10' and update_time>='2019-10-11 10:10:10' and update_time>='2019-10-12 10:10:10' and update_time='2019-10-13 10:10:10'", + startTime: timestamp(c, "2019-10-13 10:10:10"), + endTime: timestamp(c, "2019-10-13 10:10:10"), + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where update_time<='2019-10-10 10:10:10' and update_time='2019-10-13 10:10:10'", - startTime: timestamp(c, "2019-10-13 10:10:10"), - endTime: timestamp(c, "2019-10-10 10:10:10"), - skipRequest: true, + sql: "select * from information_schema.tidb_hot_regions_history where update_time<='2019-10-10 10:10:10' and update_time='2019-10-13 10:10:10'", + startTime: timestamp(c, "2019-10-13 10:10:10"), + endTime: timestamp(c, "2019-10-10 10:10:10"), + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), + skipRequest: true, }, { - sql: "select * from information_schema.tidb_hot_regions_history where update_time='2019-10-10 10:10:10' and update_time<='2019-10-13 10:10:10'", - startTime: timestamp(c, "2019-10-10 10:10:10"), - endTime: timestamp(c, "2019-10-10 10:10:10"), + sql: "select * from information_schema.tidb_hot_regions_history where update_time='2019-10-10 10:10:10' and update_time<='2019-10-13 10:10:10'", + startTime: timestamp(c, "2019-10-10 10:10:10"), + endTime: timestamp(c, "2019-10-10 10:10:10"), + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, // Test `region_id`, `store_id`, `peer_id` columns. { - sql: "select * from information_schema.tidb_hot_regions_history where region_id=100", - regionIDs: []uint64{100}, + sql: "select * from information_schema.tidb_hot_regions_history where region_id=100", + regionIDs: []uint64{100}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where 100=region_id", - regionIDs: []uint64{100}, + sql: "select * from information_schema.tidb_hot_regions_history where 100=region_id", + regionIDs: []uint64{100}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where 100=region_id or region_id=101", - regionIDs: []uint64{100, 101}, + sql: "select * from information_schema.tidb_hot_regions_history where 100=region_id or region_id=101", + regionIDs: []uint64{100, 101}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where 100=region_id or region_id=101 or region_id=102 or 103 = region_id", - regionIDs: []uint64{100, 101, 102, 103}, + sql: "select * from information_schema.tidb_hot_regions_history where 100=region_id or region_id=101 or region_id=102 or 103 = region_id", + regionIDs: []uint64{100, 101, 102, 103}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where (region_id=100 or region_id=101) and (store_id=200 or store_id=201)", - regionIDs: []uint64{100, 101}, - storeIDs: []uint64{200, 201}, + sql: "select * from information_schema.tidb_hot_regions_history where (region_id=100 or region_id=101) and (store_id=200 or store_id=201)", + regionIDs: []uint64{100, 101}, + storeIDs: []uint64{200, 201}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where region_id in (100, 101)", - regionIDs: []uint64{100, 101}, + sql: "select * from information_schema.tidb_hot_regions_history where region_id in (100, 101)", + regionIDs: []uint64{100, 101}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where region_id in (100, 101) and store_id=200", - regionIDs: []uint64{100, 101}, - storeIDs: []uint64{200}, + sql: "select * from information_schema.tidb_hot_regions_history where region_id in (100, 101) and store_id=200", + regionIDs: []uint64{100, 101}, + storeIDs: []uint64{200}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where region_id in (100, 101) and store_id in (200, 201)", - regionIDs: []uint64{100, 101}, - storeIDs: []uint64{200, 201}, + sql: "select * from information_schema.tidb_hot_regions_history where region_id in (100, 101) and store_id in (200, 201)", + regionIDs: []uint64{100, 101}, + storeIDs: []uint64{200, 201}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where region_id=100 and store_id in (200, 201)", - regionIDs: []uint64{100}, - storeIDs: []uint64{200, 201}, + sql: "select * from information_schema.tidb_hot_regions_history where region_id=100 and store_id in (200, 201)", + regionIDs: []uint64{100}, + storeIDs: []uint64{200, 201}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where region_id=100 and store_id=200", - regionIDs: []uint64{100}, - storeIDs: []uint64{200}, + sql: "select * from information_schema.tidb_hot_regions_history where region_id=100 and store_id=200", + regionIDs: []uint64{100}, + storeIDs: []uint64{200}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { sql: "select * from information_schema.tidb_hot_regions_history where region_id=100 and region_id=101", skipRequest: true, }, { - sql: "select * from information_schema.tidb_hot_regions_history where region_id=100 and region_id in (100,101)", - regionIDs: []uint64{100}, + sql: "select * from information_schema.tidb_hot_regions_history where region_id=100 and region_id in (100,101)", + regionIDs: []uint64{100}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where region_id=100 and region_id in (100,101) and store_id=200 and store_id in (200,201)", - regionIDs: []uint64{100}, - storeIDs: []uint64{200}, + sql: "select * from information_schema.tidb_hot_regions_history where region_id=100 and region_id in (100,101) and store_id=200 and store_id in (200,201)", + regionIDs: []uint64{100}, + storeIDs: []uint64{200}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { sql: "select * from information_schema.tidb_hot_regions_history where region_id=100 and region_id in (101,102)", @@ -1193,52 +1259,61 @@ func (s *extractorSuite) TestTiDBHotRegionsHistoryTableExtractor(c *C) { skipRequest: true, }, { - sql: `select * from information_schema.tidb_hot_regions_history - where region_id=100 and region_id in (100,101) + sql: `select * from information_schema.tidb_hot_regions_history + where region_id=100 and region_id in (100,101) and store_id=200 and store_id in (201,202)`, skipRequest: true, }, { - sql: "select * from information_schema.tidb_hot_regions_history where region_id in (100,101) and region_id in (101,102)", - regionIDs: []uint64{101}, + sql: "select * from information_schema.tidb_hot_regions_history where region_id in (100,101) and region_id in (101,102)", + regionIDs: []uint64{101}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: `select * from information_schema.tidb_hot_regions_history - where region_id in (100,101) - and region_id in (101,102) - and store_id in (200,201) + sql: `select * from information_schema.tidb_hot_regions_history + where region_id in (100,101) + and region_id in (101,102) + and store_id in (200,201) and store_id in (201,202)`, - regionIDs: []uint64{101}, - storeIDs: []uint64{201}, + regionIDs: []uint64{101}, + storeIDs: []uint64{201}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: `select * from information_schema.tidb_hot_regions_history - where region_id in (100,101) - and region_id in (101,102) - and store_id in (200,201) + sql: `select * from information_schema.tidb_hot_regions_history + where region_id in (100,101) + and region_id in (101,102) + and store_id in (200,201) and store_id in (201,202) and peer_id in (3000,3001) and peer_id in (3001,3002)`, - regionIDs: []uint64{101}, - storeIDs: []uint64{201}, - peerIDs: []uint64{3001}, - }, - { - sql: `select * from information_schema.tidb_hot_regions_history - where region_id in (100,101) - and region_id in (100,102) - and region_id in (102,103) + regionIDs: []uint64{101}, + storeIDs: []uint64{201}, + peerIDs: []uint64{3001}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), + }, + { + sql: `select * from information_schema.tidb_hot_regions_history + where region_id in (100,101) + and region_id in (100,102) + and region_id in (102,103) and region_id in (103,104)`, skipRequest: true, }, // Test `type` column. { sql: "select * from information_schema.tidb_hot_regions_history where type='read'", - hotRegionTypes: set.NewStringSet("read"), + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead), }, { sql: "select * from information_schema.tidb_hot_regions_history where type in('read')", - hotRegionTypes: set.NewStringSet("read"), + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead), }, { sql: "select * from information_schema.tidb_hot_regions_history where type='read' and type='write'", @@ -1246,11 +1321,11 @@ func (s *extractorSuite) TestTiDBHotRegionsHistoryTableExtractor(c *C) { }, { sql: "select * from information_schema.tidb_hot_regions_history where type in ('read', 'write')", - hotRegionTypes: set.NewStringSet("read", "write"), + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { sql: "select * from information_schema.tidb_hot_regions_history where type='read' and type in ('read', 'write')", - hotRegionTypes: set.NewStringSet("read"), + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead), }, { sql: "select * from information_schema.tidb_hot_regions_history where type in ('read') and type in ('write')", @@ -1258,33 +1333,45 @@ func (s *extractorSuite) TestTiDBHotRegionsHistoryTableExtractor(c *C) { }, // Test `is_learner`, `is_leaeder` columns. { - sql: "select * from information_schema.tidb_hot_regions_history where is_learner=1", - isLearners: []uint64{1}, + sql: "select * from information_schema.tidb_hot_regions_history where is_learner=1", + isLearners: []bool{true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where is_leader=0", - isLeaders: []uint64{0}, + sql: "select * from information_schema.tidb_hot_regions_history where is_leader=0", + isLearners: []bool{false, true}, + isLeaders: []bool{false}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where is_learner=true", - isLearners: []uint64{1}, + sql: "select * from information_schema.tidb_hot_regions_history where is_learner=true", + isLearners: []bool{true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { sql: "select * from information_schema.tidb_hot_regions_history where is_learner in(0,1)", - isLearners: []uint64{0, 1}, + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, }, { - sql: "select * from information_schema.tidb_hot_regions_history where is_learner in(true,false)", - isLearners: []uint64{0, 1}, + sql: "select * from information_schema.tidb_hot_regions_history where is_learner in(true,false)", + isLearners: []bool{false, true}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where is_learner in(3,4)", - isLearners: []uint64{3, 4}, + sql: "select * from information_schema.tidb_hot_regions_history where is_learner in(3,4)", + isLearners: []bool{false}, + isLeaders: []bool{false, true}, + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), }, { - sql: "select * from information_schema.tidb_hot_regions_history where is_learner in(3,4) and is_leader in(0,1,true,false,3,4)", - isLearners: []uint64{3, 4}, - isLeaders: []uint64{0, 1, 3, 4}, + sql: "select * from information_schema.tidb_hot_regions_history where is_learner in(3,4) and is_leader in(0,1,true,false,3,4)", + hotRegionTypes: set.NewStringSet(plannercore.HotRegionTypeRead, plannercore.HotRegionTypeWrite), + isLearners: []bool{false}, + isLeaders: []bool{false, true}, }, { sql: "select * from information_schema.tidb_hot_regions_history where is_learner=1 and is_learner=0", @@ -1322,7 +1409,6 @@ func (s *extractorSuite) TestTiDBHotRegionsHistoryTableExtractor(c *C) { if ca.hotRegionTypes.Count() > 0 { c.Assert(hotRegionsHistoryExtractor.HotRegionTypes, DeepEquals, ca.hotRegionTypes, Commentf("SQL: %v", ca.sql)) } - // ues length to avoid case uint64{} != uint64(nil) if len(ca.regionIDs) > 0 { c.Assert(hotRegionsHistoryExtractor.RegionIDs, DeepEquals, ca.regionIDs, Commentf("SQL: %v", ca.sql)) } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 55127627b7477..b964818245966 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -232,9 +232,14 @@ func CheckTableLock(ctx sessionctx.Context, is infoschema.InfoSchema, vs []visit if !config.TableLockEnabled() { return nil } + checker := lock.NewChecker(ctx, is) for i := range vs { err := checker.CheckTableLock(vs[i].db, vs[i].table, vs[i].privilege, vs[i].alterWritable) + // if table with lock-write table dropped, we can access other table, such as `rename` operation + if err == lock.ErrLockedTableDropped { + break + } if err != nil { return err } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 6505760e38b86..f9cdca27dc42f 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" @@ -548,7 +547,7 @@ func (ts *PhysicalTableScan) ResolveCorrelatedColumns() ([]*ranger.Range, error) } else { var err error pkTP := ts.Table.GetPkColInfo().FieldType - ts.Ranges, err = ranger.BuildTableRange(access, ts.SCtx().GetSessionVars().StmtCtx, &pkTP) + ts.Ranges, err = ranger.BuildTableRange(access, ts.SCtx(), &pkTP) if err != nil { return nil, err } @@ -1198,11 +1197,11 @@ func (p *PhysicalIndexScan) IsPartition() (bool, int64) { } // IsPointGetByUniqueKey checks whether is a point get by unique key. -func (p *PhysicalIndexScan) IsPointGetByUniqueKey(sc *stmtctx.StatementContext) bool { +func (p *PhysicalIndexScan) IsPointGetByUniqueKey(sctx sessionctx.Context) bool { return len(p.Ranges) == 1 && p.Index.Unique && len(p.Ranges[0].LowVal) == len(p.Index.Columns) && - p.Ranges[0].IsPoint(sc) + p.Ranges[0].IsPoint(sctx) } // PhysicalSelection represents a filter. @@ -1326,8 +1325,10 @@ const ( type PhysicalShuffleReceiverStub struct { physicalSchemaProducer - // Worker points to `executor.shuffleReceiver`. + // Receiver points to `executor.shuffleReceiver`. Receiver unsafe.Pointer + // DataSource is the PhysicalPlan of the Receiver. + DataSource PhysicalPlan } // CollectPlanStatsVersion uses to collect the statistics version of the plan. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 749b383436dcd..d1fc3ba01cdc3 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -3907,6 +3907,11 @@ func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (Plan, err b.capFlag &= ^renameView b.isCreateView = false }() + + if stmt := findStmtAsViewSchema(v); stmt != nil { + stmt.AsViewSchema = true + } + plan, err := b.Build(ctx, v.Select) if err != nil { return nil, err @@ -4402,3 +4407,18 @@ func adjustOverlongViewColname(plan LogicalPlan) { } } } + +// findStmtAsViewSchema finds the first SelectStmt as the schema for the view +func findStmtAsViewSchema(stmt ast.Node) *ast.SelectStmt { + switch x := stmt.(type) { + case *ast.CreateViewStmt: + return findStmtAsViewSchema(x.Select) + case *ast.SetOprStmt: + return findStmtAsViewSchema(x.SelectList) + case *ast.SetOprSelectList: + return findStmtAsViewSchema(x.Selects[0]) + case *ast.SelectStmt: + return x + } + return nil +} diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index c9beca4d054ec..fbf6ff47200a7 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -556,6 +556,7 @@ func newBatchPointGetPlan( handleCol *model.ColumnInfo, tbl *model.TableInfo, schema *expression.Schema, names []*types.FieldName, whereColNames []string, indexHints []*ast.IndexHint, ) *BatchPointGetPlan { + stmtCtx := ctx.GetSessionVars().StmtCtx statsInfo := &property.StatsInfo{RowCount: float64(len(patternInExpr.List))} var partitionExpr *tables.PartitionExpr if tbl.GetPartitionInfo() != nil { @@ -593,16 +594,8 @@ func newBatchPointGetPlan( if d.IsNull() { return nil } - if !checkCanConvertInPointGet(handleCol, d) { - return nil - } - intDatum, err := d.ConvertTo(ctx.GetSessionVars().StmtCtx, &handleCol.FieldType) - if err != nil { - return nil - } - // The converted result must be same as original datum - cmp, err := intDatum.CompareDatum(ctx.GetSessionVars().StmtCtx, &d) - if err != nil || cmp != 0 { + intDatum := getPointGetValue(stmtCtx, handleCol, &d) + if intDatum == nil { return nil } handles[i] = kv.IntHandle(intDatum.GetInt64()) @@ -686,12 +679,14 @@ func newBatchPointGetPlan( permIndex := permutations[index] switch innerX := inner.(type) { case *driver.ValueExpr: - if !checkCanConvertInPointGet(colInfos[index], innerX.Datum) { + dval := getPointGetValue(stmtCtx, colInfos[index], &innerX.Datum) + if dval == nil { return nil } values[permIndex] = innerX.Datum case *driver.ParamMarkerExpr: - if !checkCanConvertInPointGet(colInfos[index], innerX.Datum) { + dval := getPointGetValue(stmtCtx, colInfos[index], &innerX.Datum) + if dval == nil { return nil } values[permIndex] = innerX.Datum @@ -706,18 +701,20 @@ func newBatchPointGetPlan( if len(whereColNames) != 1 { return nil } - if !checkCanConvertInPointGet(colInfos[0], x.Datum) { + dval := getPointGetValue(stmtCtx, colInfos[0], &x.Datum) + if dval == nil { return nil } - values = []types.Datum{x.Datum} + values = []types.Datum{*dval} case *driver.ParamMarkerExpr: if len(whereColNames) != 1 { return nil } - if !checkCanConvertInPointGet(colInfos[0], x.Datum) { + dval := getPointGetValue(stmtCtx, colInfos[0], &x.Datum) + if dval == nil { return nil } - values = []types.Datum{x.Datum} + values = []types.Datum{*dval} valuesParams = []*driver.ParamMarkerExpr{x} default: return nil @@ -1229,6 +1226,23 @@ func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, return nil, false } +func getPointGetValue(stmtCtx *stmtctx.StatementContext, col *model.ColumnInfo, d *types.Datum) *types.Datum { + if !checkCanConvertInPointGet(col, *d) { + return nil + } + dVal, err := d.ConvertTo(stmtCtx, &col.FieldType) + if err != nil { + return nil + } + // The converted result must be same as original datum. + // Compare them based on the dVal's type. + cmp, err := dVal.CompareDatum(stmtCtx, d) + if err != nil || cmp != 0 { + return nil + } + return &dVal +} + func checkCanConvertInPointGet(col *model.ColumnInfo, d types.Datum) bool { kind := d.Kind() switch col.FieldType.EvalType() { diff --git a/planner/core/point_get_plan_test.go b/planner/core/point_get_plan_test.go index 9d426ed719cfc..300370b60dee6 100644 --- a/planner/core/point_get_plan_test.go +++ b/planner/core/point_get_plan_test.go @@ -899,6 +899,36 @@ func (s *testPointGetSuite) TestIssue18042(c *C) { tk.MustExec("drop table t") } +func (s *testPointGetSuite) TestIssue26638(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a float, unique index uidx(a));") + tk.MustExec("insert into t values(9.46347e37), (1.1);") + // If we do not define the precision for the float type. We can not use the equal/ in conditions to find the result. We can only use like to find the result. There is no such restriction for the double type. + tk.MustQuery("explain format='brief' select * from t where a = 9.46347e37;").Check(testkit.Rows("TableDual 0.00 root rows:0")) + tk.MustQuery("explain format='brief' select * from t where a in (-1.56018e38, -1.96716e38, 9.46347e37);").Check(testkit.Rows("TableDual 0.00 root rows:0")) + tk.MustQuery("explain format='brief' select * from t where a in (1.1, 9.46347e37);").Check(testkit.Rows("TableDual 0.00 root rows:0")) + tk.MustExec("prepare stmt from 'select * from t where a in (?, ?, ?);';") + tk.MustExec("prepare stmt1 from 'select * from t where a = ?;';") + tk.MustExec("prepare stmt2 from 'select * from t where a in (?, ?);';") + tk.MustExec("set @a=-1.56018e38, @b=-1.96716e38, @c=9.46347e37, @d=1.1, @e=0, @f=-1, @g=1, @h=2, @i=-1.1;") + tk.MustQuery("execute stmt using @a,@b,@c;").Check(testkit.Rows()) + tk.MustQuery("execute stmt1 using @c;").Check(testkit.Rows()) + tk.MustQuery("execute stmt2 using @c, @d;").Check(testkit.Rows()) + tk.MustExec("drop table if exists t2;") + tk.MustExec("create table t2(a float, b float, c float, primary key(a, b, c));") + tk.MustExec("insert into t2 values(-1, 0, 1), (-1.1, 0, 1.1), (-1.56018e38, -1.96716e38, 9.46347e37), (0, 1, 2);") + tk.MustQuery("explain format='brief' select * from t2 where (a, b, c) in ((-1.1, 0, 1.1), (-1.56018e38, -1.96716e38, 9.46347e37));").Check(testkit.Rows("TableDual 0.00 root rows:0")) + tk.MustQuery("select * from t2 where (a, b, c) in ((-1.1, 0, 1.1), (-1.56018e38, -1.96716e38, 9.46347e37), (-1, 0, 1));").Check(testkit.Rows("-1 0 1")) + tk.MustQuery("select * from t2 where (a, b, c) in ((-1.1, 0, 1.1), (-1.56018e38, -1.96716e38, 9.46347e37), (0, 1, 2));").Check(testkit.Rows("0 1 2")) + tk.MustExec("prepare stmt3 from 'select * from t2 where (a, b, c) in ((?, ?, ?), (?, ?, ?));';") + tk.MustExec("prepare stmt4 from 'select * from t2 where (a, b, c) in ((?, ?, ?), (?, ?, ?), (?, ?, ?));';") + tk.MustQuery("execute stmt3 using @i,@e,@d,@a,@b,@c;").Check(testkit.Rows()) + tk.MustQuery("execute stmt4 using @i,@e,@d,@a,@b,@c,@f,@e,@g;").Check(testkit.Rows("-1 0 1")) + tk.MustQuery("execute stmt4 using @i,@e,@d,@a,@b,@c,@e,@g,@h;").Check(testkit.Rows("0 1 2")) +} + func (s *testPointGetSuite) TestIssue23511(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 49dde61367afd..bb57b0fac33da 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -140,7 +140,7 @@ func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl tabl ranges := detachedResult.Ranges used := make([]int, 0, len(ranges)) for _, r := range ranges { - if r.IsPointNullable(ctx.GetSessionVars().StmtCtx) { + if r.IsPointNullable(ctx) { if !r.HighVal[0].IsNull() { if len(r.HighVal) != len(partIdx) { used = []int{-1} @@ -473,7 +473,7 @@ func (l *listPartitionPruner) locateColumnPartitionsByCondition(cond expression. return nil, true, nil } var locations []tables.ListPartitionLocation - if r.IsPointNullable(sc) { + if r.IsPointNullable(l.ctx) { location, err := colPrune.LocatePartition(sc, r.HighVal[0]) if types.ErrOverflow.Equal(err) { return nil, true, nil // return full-scan if over-flow @@ -555,7 +555,7 @@ func (l *listPartitionPruner) findUsedListPartitions(conds []expression.Expressi } used := make(map[int]struct{}, len(ranges)) for _, r := range ranges { - if r.IsPointNullable(l.ctx.GetSessionVars().StmtCtx) { + if r.IsPointNullable(l.ctx) { if len(r.HighVal) != len(exprCols) { return l.fullRange, nil } diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index 793dbc0d31cbb..f59fedc25ba28 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -288,10 +288,14 @@ func (p *LogicalProjection) appendExpr(expr expression.Expression) *expression.C col := &expression.Column{ UniqueID: p.ctx.GetSessionVars().AllocPlanColumnID(), - RetType: expr.GetType(), + RetType: expr.GetType().Clone(), } col.SetCoercibility(expr.Coercibility()) p.schema.Append(col) + // reset ParseToJSONFlag in order to keep the flag away from json column + if col.GetType().Tp == mysql.TypeJSON { + col.GetType().Flag &= ^mysql.ParseToJSONFlag + } return col } diff --git a/planner/core/stats.go b/planner/core/stats.go index 7f0d62da1dc49..ac7d6e99a4f8c 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -284,7 +284,7 @@ func (ds *DataSource) derivePathStatsAndTryHeuristics() error { selected = path break } - if path.OnlyPointRange(ds.SCtx().GetSessionVars().StmtCtx) { + if path.OnlyPointRange(ds.SCtx()) { if path.IsTablePath() || path.Index.Unique { if path.IsSingleScan { selected = path @@ -477,11 +477,10 @@ func (ts *LogicalTableScan) DeriveStats(childStats []*property.StatsInfo, selfSc ts.AccessConds[i] = expression.PushDownNot(ts.ctx, expr) } ts.stats = ts.Source.deriveStatsByFilter(ts.AccessConds, nil) - sc := ts.SCtx().GetSessionVars().StmtCtx // ts.Handle could be nil if PK is Handle, and PK column has been pruned. // TODO: support clustered index. if ts.HandleCols != nil { - ts.Ranges, err = ranger.BuildTableRange(ts.AccessConds, sc, ts.HandleCols.GetCol(0).RetType) + ts.Ranges, err = ranger.BuildTableRange(ts.AccessConds, ts.ctx, ts.HandleCols.GetCol(0).RetType) } else { isUnsigned := false if ts.Source.tableInfo.PKIsHandle { @@ -633,7 +632,7 @@ func (ds *DataSource) accessPathsForConds(conditions []expression.Expression, us continue } // If we have point or empty range, just remove other possible paths. - if len(path.Ranges) == 0 || path.OnlyPointRange(ds.SCtx().GetSessionVars().StmtCtx) { + if len(path.Ranges) == 0 || path.OnlyPointRange(ds.SCtx()) { if len(results) == 0 { results = append(results, path) } else { @@ -658,7 +657,7 @@ func (ds *DataSource) accessPathsForConds(conditions []expression.Expression, us continue } // If we have empty range, or point range on unique index, just remove other possible paths. - if len(path.Ranges) == 0 || (path.OnlyPointRange(ds.SCtx().GetSessionVars().StmtCtx) && path.Index.Unique) { + if len(path.Ranges) == 0 || (path.OnlyPointRange(ds.SCtx()) && path.Index.Unique) { if len(results) == 0 { results = append(results, path) } else { diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 47fb9e613358d..6c946cdac4d2d 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -675,19 +675,19 @@ { "SQL": "select /*+ USE_INDEX() */ * from t1, t2 where t1.a=t2.a", "Warnings": [ - "[parser:1064]You have an error in your SQL syntax; check the manual that corresponds to your TiDB version for the right syntax to use [parser:8064]Optimizer hint syntax error at line 1 column 22 near \") */\" " + "[parser:1064]Optimizer hint syntax error at line 1 column 22 near \") */\" " ] }, { "SQL": "select /*+ IGNORE_INDEX() */ * from t1, t2 where t1.a=t2.a", "Warnings": [ - "[parser:1064]You have an error in your SQL syntax; check the manual that corresponds to your TiDB version for the right syntax to use [parser:8064]Optimizer hint syntax error at line 1 column 25 near \") */\" " + "[parser:1064]Optimizer hint syntax error at line 1 column 25 near \") */\" " ] }, { "SQL": "select /*+ USE_INDEX_MERGE() */ * from t1, t2 where t1.a=t2.a", "Warnings": [ - "[parser:1064]You have an error in your SQL syntax; check the manual that corresponds to your TiDB version for the right syntax to use [parser:8064]Optimizer hint syntax error at line 1 column 28 near \") */\" " + "[parser:1064]Optimizer hint syntax error at line 1 column 28 near \") */\" " ] } ] diff --git a/planner/core/util.go b/planner/core/util.go index 517bb24bf7864..480c4bdfabd0c 100644 --- a/planner/core/util.go +++ b/planner/core/util.go @@ -308,6 +308,19 @@ func extractStringFromUint64Slice(slice []uint64) string { return strings.Join(l, ",") } +// extractStringFromBoolSlice helps extract string info from bool slice. +func extractStringFromBoolSlice(slice []bool) string { + if len(slice) < 1 { + return "" + } + l := make([]string, 0, len(slice)) + for _, k := range slice { + l = append(l, fmt.Sprintf(`%t`, k)) + } + sort.Strings(l) + return strings.Join(l, ",") +} + func tableHasDirtyContent(ctx sessionctx.Context, tableInfo *model.TableInfo) bool { pi := tableInfo.GetPartitionInfo() if pi == nil { diff --git a/planner/optimize.go b/planner/optimize.go index 68c0a12d871f8..363c3e6f5374a 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -160,9 +160,10 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in useBinding = false } } - if useBinding && sessVars.SelectLimit != math.MaxUint64 { - sessVars.StmtCtx.AppendWarning(errors.New("sql_select_limit is set, ignore SQL bindings")) - useBinding = false + if ok { + // add the extra Limit after matching the bind record + stmtNode = plannercore.TryAddExtraLimit(sctx, stmtNode) + node = stmtNode } var names types.NameSlice @@ -234,7 +235,8 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in defer func() { sessVars.StmtCtx.StmtHints = savedStmtHints }() - if sessVars.EvolvePlanBaselines && bestPlanFromBind != nil { + if sessVars.EvolvePlanBaselines && bestPlanFromBind != nil && + sessVars.SelectLimit == math.MaxUint64 { // do not evolve this query if sql_select_limit is enabled // Check bestPlanFromBind firstly to avoid nil stmtNode. if _, ok := stmtNode.(*ast.SelectStmt); ok && !bindRecord.Bindings[0].Hint.ContainTableHint(plannercore.HintReadFromStorage) { sessVars.StmtCtx.StmtHints = originStmtHints diff --git a/planner/util/path.go b/planner/util/path.go index 694ea781959aa..1d895a69bf2e0 100644 --- a/planner/util/path.go +++ b/planner/util/path.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/ranger" @@ -151,11 +150,11 @@ func isColEqCorColOrConstant(ctx sessionctx.Context, filter expression.Expressio } // OnlyPointRange checks whether each range is a point(no interval range exists). -func (path *AccessPath) OnlyPointRange(sc *stmtctx.StatementContext) bool { +func (path *AccessPath) OnlyPointRange(sctx sessionctx.Context) bool { noIntervalRange := true if path.IsIntHandlePath { for _, ran := range path.Ranges { - if !ran.IsPoint(sc) { + if !ran.IsPoint(sctx) { noIntervalRange = false break } @@ -165,7 +164,7 @@ func (path *AccessPath) OnlyPointRange(sc *stmtctx.StatementContext) bool { haveNullVal := false for _, ran := range path.Ranges { // Not point or the not full matched. - if !ran.IsPoint(sc) || len(ran.HighVal) != len(path.Index.Columns) { + if !ran.IsPoint(sctx) || len(ran.HighVal) != len(path.Index.Columns) { noIntervalRange = false break } diff --git a/privilege/privileges/cache.go b/privilege/privileges/cache.go index 0652cd9ff8d84..150a47d896403 100644 --- a/privilege/privileges/cache.go +++ b/privilege/privileges/cache.go @@ -1257,7 +1257,8 @@ func (p *MySQLPrivilege) showGrants(user, host string, roles []*auth.RoleIdentit // A map of "DB.Table" => Priv(col1, col2 ...) sortFromIdx = len(gs) columnPrivTable := make(map[string]privOnColumns) - for _, record := range p.ColumnsPriv { + for i := range p.ColumnsPriv { + record := p.ColumnsPriv[i] if !collectColumnGrant(&record, user, host, columnPrivTable) { for _, r := range allRoles { collectColumnGrant(&record, r.Username, r.Hostname, columnPrivTable) diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 8530cd47fae65..6d9c6dd0d62bc 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -2668,6 +2668,7 @@ func TestGrantCreateTmpTables(t *testing.T) { tk.MustExec("CREATE TABLE create_tmp_table_table (a int)") tk.MustExec("GRANT CREATE TEMPORARY TABLES on create_tmp_table_db.* to u1") tk.MustExec("GRANT CREATE TEMPORARY TABLES on *.* to u1") + tk.MustGetErrCode("GRANT CREATE TEMPORARY TABLES on create_tmp_table_db.tmp to u1", mysql.ErrIllegalGrantForTable) // Must set a session user to avoid null pointer dereference tk.Session().Auth(&auth.UserIdentity{ Username: "root", diff --git a/server/column.go b/server/column.go index a142221b6acdc..2ef6323d83b5f 100644 --- a/server/column.go +++ b/server/column.go @@ -15,6 +15,7 @@ package server import ( + "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" ) @@ -39,7 +40,7 @@ type ColumnInfo struct { // Dump dumps ColumnInfo to bytes. func (column *ColumnInfo) Dump(buffer []byte, d *resultEncoder) []byte { if d == nil { - d = &resultEncoder{} + d = newResultEncoder(charset.CharsetUTF8MB4) } nameDump, orgnameDump := []byte(column.Name), []byte(column.OrgName) if len(nameDump) > maxColumnNameSize { diff --git a/server/conn.go b/server/conn.go index 8658b8fe61bc8..0277bd40a49be 100644 --- a/server/conn.go +++ b/server/conn.go @@ -707,7 +707,7 @@ func (cc *clientConn) readOptionalSSLRequestAndHandshakeResponse(ctx context.Con func (cc *clientConn) handleAuthPlugin(ctx context.Context, resp *handshakeResponse41) error { if resp.Capability&mysql.ClientPluginAuth > 0 { - newAuth, err := cc.checkAuthPlugin(ctx, &resp.AuthPlugin) + newAuth, err := cc.checkAuthPlugin(ctx, resp) if err != nil { logutil.Logger(ctx).Warn("failed to check the user authplugin", zap.Error(err)) } @@ -830,7 +830,7 @@ func (cc *clientConn) openSessionAndDoAuth(authData []byte, authPlugin string) e } // Check if the Authentication Plugin of the server, client and user configuration matches -func (cc *clientConn) checkAuthPlugin(ctx context.Context, authPlugin *string) ([]byte, error) { +func (cc *clientConn) checkAuthPlugin(ctx context.Context, resp *handshakeResponse41) ([]byte, error) { // Open a context unless this was done before. if cc.ctx == nil { err := cc.openSession() @@ -839,12 +839,21 @@ func (cc *clientConn) checkAuthPlugin(ctx context.Context, authPlugin *string) ( } } - userplugin, err := cc.ctx.AuthPluginForUser(&auth.UserIdentity{Username: cc.user, Hostname: cc.peerHost}) + authData := resp.Auth + hasPassword := "YES" + if len(authData) == 0 { + hasPassword = "NO" + } + host, _, err := cc.PeerHost(hasPassword) + if err != nil { + return nil, err + } + userplugin, err := cc.ctx.AuthPluginForUser(&auth.UserIdentity{Username: cc.user, Hostname: host}) if err != nil { return nil, err } if userplugin == mysql.AuthSocket { - *authPlugin = mysql.AuthSocket + resp.AuthPlugin = mysql.AuthSocket user, err := user.LookupId(fmt.Sprint(cc.socketCredUID)) if err != nil { return nil, err @@ -854,7 +863,7 @@ func (cc *clientConn) checkAuthPlugin(ctx context.Context, authPlugin *string) ( if len(userplugin) == 0 { logutil.Logger(ctx).Warn("No user plugin set, assuming MySQL Native Password", zap.String("user", cc.user), zap.String("host", cc.peerHost)) - *authPlugin = mysql.AuthNativePassword + resp.AuthPlugin = mysql.AuthNativePassword return nil, nil } @@ -863,12 +872,12 @@ func (cc *clientConn) checkAuthPlugin(ctx context.Context, authPlugin *string) ( // or if the authentication method send by the server doesn't match the authentication // method send by the client (*authPlugin) then we need to switch the authentication // method to match the one configured for that specific user. - if (cc.authPlugin != userplugin) || (cc.authPlugin != *authPlugin) { + if (cc.authPlugin != userplugin) || (cc.authPlugin != resp.AuthPlugin) { authData, err := cc.authSwitchRequest(ctx, userplugin) if err != nil { return nil, err } - *authPlugin = userplugin + resp.AuthPlugin = userplugin return authData, nil } diff --git a/server/conn_test.go b/server/conn_test.go index 06b4b3a38c926..34ec75124a828 100644 --- a/server/conn_test.go +++ b/server/conn_test.go @@ -943,3 +943,47 @@ func TestHandleAuthPlugin(t *testing.T) { err = cc.handleAuthPlugin(ctx, &resp) require.NoError(t, err) } + +func TestAuthPlugin2(t *testing.T) { + + t.Parallel() + + store, clean := testkit.CreateMockStore(t) + defer clean() + + cfg := newTestConfig() + cfg.Socket = "" + cfg.Port = 0 + cfg.Status.StatusPort = 0 + + drv := NewTiDBDriver(store) + srv, err := NewServer(cfg, drv) + require.NoError(t, err) + + cc := &clientConn{ + connectionID: 1, + alloc: arena.NewAllocator(1024), + chunkAlloc: chunk.NewAllocator(), + pkt: &packetIO{ + bufWriter: bufio.NewWriter(bytes.NewBuffer(nil)), + }, + server: srv, + user: "root", + } + ctx := context.Background() + se, _ := session.CreateSession4Test(store) + tc := &TiDBContext{ + Session: se, + stmts: make(map[int]*TiDBStatement), + } + cc.ctx = tc + + resp := handshakeResponse41{ + Capability: mysql.ClientProtocol41 | mysql.ClientPluginAuth, + } + + cc.isUnixSocket = true + _, err = cc.checkAuthPlugin(ctx, &resp) + require.NoError(t, err) + +} diff --git a/server/main_test.go b/server/main_test.go index 4c3969da1f506..155d9f9b7294e 100644 --- a/server/main_test.go +++ b/server/main_test.go @@ -21,6 +21,7 @@ import ( "testing" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/util/testbridge" "github.com/tikv/client-go/v2/tikv" @@ -30,12 +31,16 @@ import ( func TestMain(m *testing.M) { testbridge.WorkaroundGoCheckFlags() + runInGoTest = true // flag for NewServer to known it is running in test environment + // AsyncCommit will make DDL wait 2.5s before changing to the next state. // Set schema lease to avoid it from making CI slow. session.SetSchemaLease(0) tikv.EnableFailpoints() + metrics.RegisterMetrics() + // sanity check: the global config should not be changed by other pkg init function. // see also https://github.com/pingcap/tidb/issues/22162 defaultConfig := config.NewConfig() diff --git a/server/server.go b/server/server.go index e656569a14a77..34f71b6ff06e3 100644 --- a/server/server.go +++ b/server/server.go @@ -32,7 +32,6 @@ package server import ( "context" "crypto/tls" - "flag" "fmt" "math/rand" "net" @@ -88,7 +87,6 @@ func init() { if err != nil { osVersion = "" } - runInGoTest = flag.Lookup("test.v") != nil || flag.Lookup("check.v") != nil } var ( diff --git a/server/server_test.go b/server/server_test.go index 82e4dd6ab883e..210e58caed3f8 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -32,14 +32,15 @@ import ( "time" "github.com/go-sql-driver/mysql" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" tmysql "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/versioninfo" + "github.com/stretchr/testify/require" "go.uber.org/zap" ) @@ -47,11 +48,6 @@ var ( regression = true ) -func TestT(t *testing.T) { - CustomVerboseFlag = true - TestingT(t) -} - type configOverrider func(*mysql.Config) // testServerClient config server connect parameters and provider several @@ -108,343 +104,304 @@ func (cli *testServerClient) getDSN(overriders ...configOverrider) string { } // runTests runs tests using the default database `test`. -func (cli *testServerClient) runTests(c *C, overrider configOverrider, tests ...func(dbt *DBTest)) { +func (cli *testServerClient) runTests(t *testing.T, overrider configOverrider, tests ...func(dbt *testkit.DBTestKit)) { db, err := sql.Open("mysql", cli.getDSN(overrider)) - c.Assert(err, IsNil, Commentf("Error connecting")) + require.NoErrorf(t, err, "Error connecting") defer func() { err := db.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() - dbt := &DBTest{c, db} + dbt := testkit.NewDBTestKit(t, db) for _, test := range tests { test(dbt) } } // runTestsOnNewDB runs tests using a specified database which will be created before the test and destroyed after the test. -func (cli *testServerClient) runTestsOnNewDB(c *C, overrider configOverrider, dbName string, tests ...func(dbt *DBTest)) { +func (cli *testServerClient) runTestsOnNewDB(t *testing.T, overrider configOverrider, dbName string, tests ...func(dbt *testkit.DBTestKit)) { dsn := cli.getDSN(overrider, func(config *mysql.Config) { config.DBName = "" }) db, err := sql.Open("mysql", dsn) - c.Assert(err, IsNil, Commentf("Error connecting")) + require.NoErrorf(t, err, "Error connecting") defer func() { err := db.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() _, err = db.Exec(fmt.Sprintf("DROP DATABASE IF EXISTS `%s`;", dbName)) if err != nil { fmt.Println(err) } - c.Assert(err, IsNil, Commentf("Error drop database %s: %s", dbName, err)) + require.NoErrorf(t, err, "Error drop database %s: %s", dbName, err) _, err = db.Exec(fmt.Sprintf("CREATE DATABASE `%s`;", dbName)) - c.Assert(err, IsNil, Commentf("Error create database %s: %s", dbName, err)) + require.NoErrorf(t, err, "Error create database %s: %s", dbName, err) defer func() { _, err = db.Exec(fmt.Sprintf("DROP DATABASE IF EXISTS `%s`;", dbName)) - c.Assert(err, IsNil, Commentf("Error drop database %s: %s", dbName, err)) + require.NoErrorf(t, err, "Error drop database %s: %s", dbName, err) }() _, err = db.Exec(fmt.Sprintf("USE `%s`;", dbName)) - c.Assert(err, IsNil, Commentf("Error use database %s: %s", dbName, err)) + require.NoErrorf(t, err, "Error use database %s: %s", dbName, err) - dbt := &DBTest{c, db} + dbt := testkit.NewDBTestKit(t, db) for _, test := range tests { test(dbt) // to fix : no db selected - _, _ = dbt.db.Exec("DROP TABLE IF EXISTS test") + _, _ = dbt.GetDB().Exec("DROP TABLE IF EXISTS test") } } -type DBTest struct { - *C - db *sql.DB -} - -func (dbt *DBTest) mustPrepare(query string) *sql.Stmt { - stmt, err := dbt.db.Prepare(query) - dbt.Assert(err, IsNil, Commentf("Prepare %s", query)) - return stmt -} - -func (dbt *DBTest) mustExecPrepared(stmt *sql.Stmt, args ...interface{}) sql.Result { - res, err := stmt.Exec(args...) - dbt.Assert(err, IsNil, Commentf("Execute prepared with args: %s", args)) - return res -} - -func (dbt *DBTest) mustQueryPrepared(stmt *sql.Stmt, args ...interface{}) *sql.Rows { - rows, err := stmt.Query(args...) - dbt.Assert(err, IsNil, Commentf("Query prepared with args: %s", args)) - return rows -} - -func (dbt *DBTest) mustExec(query string, args ...interface{}) (res sql.Result) { - res, err := dbt.db.Exec(query, args...) - dbt.Assert(err, IsNil, Commentf("Exec %s", query)) - return res -} - -func (dbt *DBTest) mustQuery(query string, args ...interface{}) (rows *sql.Rows) { - rows, err := dbt.db.Query(query, args...) - dbt.Assert(err, IsNil, Commentf("Query %s", query)) - return rows -} - -func (dbt *DBTest) mustQueryRows(query string, args ...interface{}) { - rows := dbt.mustQuery(query, args...) - dbt.Assert(rows.Next(), IsTrue) - rows.Close() -} - -func (cli *testServerClient) runTestRegression(c *C, overrider configOverrider, dbName string) { - cli.runTestsOnNewDB(c, overrider, dbName, func(dbt *DBTest) { +func (cli *testServerClient) runTestRegression(t *testing.T, overrider configOverrider, dbName string) { + cli.runTestsOnNewDB(t, overrider, dbName, func(dbt *testkit.DBTestKit) { // Show the user - dbt.mustExec("select user()") + dbt.MustExec("select user()") // Create Table - dbt.mustExec("CREATE TABLE test (val TINYINT)") + dbt.MustExec("CREATE TABLE test (val TINYINT)") // Test for unexpected data var out bool - rows := dbt.mustQuery("SELECT * FROM test") - dbt.Assert(rows.Next(), IsFalse, Commentf("unexpected data in empty table")) + rows := dbt.MustQuery("SELECT * FROM test") + require.Falsef(t, rows.Next(), "unexpected data in empty table") // Create Data - res := dbt.mustExec("INSERT INTO test VALUES (1)") + res := dbt.MustExec("INSERT INTO test VALUES (1)") // res := dbt.mustExec("INSERT INTO test VALUES (?)", 1) count, err := res.RowsAffected() - dbt.Assert(err, IsNil) - dbt.Check(count, Equals, int64(1)) + require.NoError(t, err) + require.Equal(t, int64(1), count) id, err := res.LastInsertId() - dbt.Assert(err, IsNil) - dbt.Check(id, Equals, int64(0)) + require.NoError(t, err) + require.Equal(t, int64(0), id) // Read - rows = dbt.mustQuery("SELECT val FROM test") + rows = dbt.MustQuery("SELECT val FROM test") if rows.Next() { err = rows.Scan(&out) - c.Assert(err, IsNil) - dbt.Check(out, IsTrue) - dbt.Check(rows.Next(), IsFalse, Commentf("unexpected data")) + require.NoError(t, err) + require.True(t, out) + require.Falsef(t, rows.Next(), "unexpected data") } else { - dbt.Error("no data") + require.Fail(t, "no data") } rows.Close() // Update - res = dbt.mustExec("UPDATE test SET val = 0 WHERE val = ?", 1) + res = dbt.MustExec("UPDATE test SET val = 0 WHERE val = ?", 1) count, err = res.RowsAffected() - dbt.Assert(err, IsNil) - dbt.Check(count, Equals, int64(1)) + require.NoError(t, err) + require.Equal(t, int64(1), count) // Check Update - rows = dbt.mustQuery("SELECT val FROM test") + rows = dbt.MustQuery("SELECT val FROM test") if rows.Next() { err = rows.Scan(&out) - c.Assert(err, IsNil) - dbt.Check(out, IsFalse) - dbt.Check(rows.Next(), IsFalse, Commentf("unexpected data")) + require.NoError(t, err) + require.False(t, out) + require.Falsef(t, rows.Next(), "unexpected data") } else { - dbt.Error("no data") + require.Fail(t, "no data") } rows.Close() // Delete - res = dbt.mustExec("DELETE FROM test WHERE val = 0") + res = dbt.MustExec("DELETE FROM test WHERE val = 0") // res = dbt.mustExec("DELETE FROM test WHERE val = ?", 0) count, err = res.RowsAffected() - dbt.Assert(err, IsNil) - dbt.Check(count, Equals, int64(1)) + require.NoError(t, err) + require.Equal(t, int64(1), count) // Check for unexpected rows - res = dbt.mustExec("DELETE FROM test") + res = dbt.MustExec("DELETE FROM test") count, err = res.RowsAffected() - dbt.Assert(err, IsNil) - dbt.Check(count, Equals, int64(0)) + require.NoError(t, err) + require.Equal(t, int64(0), count) - dbt.mustQueryRows("SELECT 1") + dbt.MustQueryRows("SELECT 1") var b = make([]byte, 0) - if err := dbt.db.QueryRow("SELECT ?", b).Scan(&b); err != nil { - dbt.Fatal(err) + if err := dbt.GetDB().QueryRow("SELECT ?", b).Scan(&b); err != nil { + t.Fatal(err) } if b == nil { - dbt.Error("nil echo from non-nil input") + require.Fail(t, "nil echo from non-nil input") } }) } -func (cli *testServerClient) runTestPrepareResultFieldType(t *C) { +func (cli *testServerClient) runTestPrepareResultFieldType(t *testing.T) { var param int64 = 83 - cli.runTests(t, nil, func(dbt *DBTest) { - stmt, err := dbt.db.Prepare(`SELECT ?`) + cli.runTests(t, nil, func(dbt *testkit.DBTestKit) { + stmt, err := dbt.GetDB().Prepare(`SELECT ?`) if err != nil { - dbt.Fatal(err) + t.Fatal(err) } defer stmt.Close() row := stmt.QueryRow(param) var result int64 err = row.Scan(&result) if err != nil { - dbt.Fatal(err) + t.Fatal(err) } if result != param { - dbt.Fatal("Unexpected result value") + t.Fatal("Unexpected result value") } }) } -func (cli *testServerClient) runTestSpecialType(t *C) { - cli.runTestsOnNewDB(t, nil, "SpecialType", func(dbt *DBTest) { - dbt.mustExec("create table test (a decimal(10, 5), b datetime, c time, d bit(8))") - dbt.mustExec("insert test values (1.4, '2012-12-21 12:12:12', '4:23:34', b'1000')") - rows := dbt.mustQuery("select * from test where a > ?", 0) - t.Assert(rows.Next(), IsTrue) +func (cli *testServerClient) runTestSpecialType(t *testing.T) { + cli.runTestsOnNewDB(t, nil, "SpecialType", func(dbt *testkit.DBTestKit) { + dbt.MustExec("create table test (a decimal(10, 5), b datetime, c time, d bit(8))") + dbt.MustExec("insert test values (1.4, '2012-12-21 12:12:12', '4:23:34', b'1000')") + rows := dbt.MustQuery("select * from test where a > ?", 0) + require.True(t, rows.Next()) var outA float64 var outB, outC string var outD []byte err := rows.Scan(&outA, &outB, &outC, &outD) - t.Assert(err, IsNil) - t.Assert(outA, Equals, 1.4) - t.Assert(outB, Equals, "2012-12-21 12:12:12") - t.Assert(outC, Equals, "04:23:34") - t.Assert(outD, BytesEquals, []byte{8}) + require.NoError(t, err) + require.Equal(t, 1.4, outA) + require.Equal(t, "2012-12-21 12:12:12", outB) + require.Equal(t, "04:23:34", outC) + require.Equal(t, []byte{8}, outD) }) } -func (cli *testServerClient) runTestClientWithCollation(t *C) { +func (cli *testServerClient) runTestClientWithCollation(t *testing.T) { cli.runTests(t, func(config *mysql.Config) { config.Collation = "utf8mb4_general_ci" - }, func(dbt *DBTest) { + }, func(dbt *testkit.DBTestKit) { var name, charset, collation string // check session variable collation_connection - rows := dbt.mustQuery("show variables like 'collation_connection'") - t.Assert(rows.Next(), IsTrue) + rows := dbt.MustQuery("show variables like 'collation_connection'") + require.True(t, rows.Next()) + err := rows.Scan(&name, &collation) - t.Assert(err, IsNil) - t.Assert(collation, Equals, "utf8mb4_general_ci") + require.NoError(t, err) + require.Equal(t, "utf8mb4_general_ci", collation) // check session variable character_set_client - rows = dbt.mustQuery("show variables like 'character_set_client'") - t.Assert(rows.Next(), IsTrue) + rows = dbt.MustQuery("show variables like 'character_set_client'") + require.True(t, rows.Next()) err = rows.Scan(&name, &charset) - t.Assert(err, IsNil) - t.Assert(charset, Equals, "utf8mb4") + require.NoError(t, err) + require.Equal(t, "utf8mb4", charset) // check session variable character_set_results - rows = dbt.mustQuery("show variables like 'character_set_results'") - t.Assert(rows.Next(), IsTrue) + rows = dbt.MustQuery("show variables like 'character_set_results'") + require.True(t, rows.Next()) err = rows.Scan(&name, &charset) - t.Assert(err, IsNil) - t.Assert(charset, Equals, "utf8mb4") + require.NoError(t, err) + require.Equal(t, "utf8mb4", charset) // check session variable character_set_connection - rows = dbt.mustQuery("show variables like 'character_set_connection'") - t.Assert(rows.Next(), IsTrue) + rows = dbt.MustQuery("show variables like 'character_set_connection'") + require.True(t, rows.Next()) err = rows.Scan(&name, &charset) - t.Assert(err, IsNil) - t.Assert(charset, Equals, "utf8mb4") + require.NoError(t, err) + require.Equal(t, "utf8mb4", charset) }) } -func (cli *testServerClient) runTestPreparedString(t *C) { - cli.runTestsOnNewDB(t, nil, "PreparedString", func(dbt *DBTest) { - dbt.mustExec("create table test (a char(10), b char(10))") - dbt.mustExec("insert test values (?, ?)", "abcdeabcde", "abcde") - rows := dbt.mustQuery("select * from test where 1 = ?", 1) - t.Assert(rows.Next(), IsTrue) +func (cli *testServerClient) runTestPreparedString(t *testing.T) { + cli.runTestsOnNewDB(t, nil, "PreparedString", func(dbt *testkit.DBTestKit) { + dbt.MustExec("create table test (a char(10), b char(10))") + dbt.MustExec("insert test values (?, ?)", "abcdeabcde", "abcde") + rows := dbt.MustQuery("select * from test where 1 = ?", 1) + require.True(t, rows.Next()) var outA, outB string err := rows.Scan(&outA, &outB) - t.Assert(err, IsNil) - t.Assert(outA, Equals, "abcdeabcde") - t.Assert(outB, Equals, "abcde") + require.NoError(t, err) + require.Equal(t, "abcdeabcde", outA) + require.Equal(t, "abcde", outB) }) } // runTestPreparedTimestamp does not really cover binary timestamp format, because MySQL driver in golang // does not use this format. MySQL driver in golang will convert the timestamp to a string. // This case guarantees it could work. -func (cli *testServerClient) runTestPreparedTimestamp(t *C) { - cli.runTestsOnNewDB(t, nil, "prepared_timestamp", func(dbt *DBTest) { - dbt.mustExec("create table test (a timestamp, b time)") - dbt.mustExec("set time_zone='+00:00'") - insertStmt := dbt.mustPrepare("insert test values (?, ?)") +func (cli *testServerClient) runTestPreparedTimestamp(t *testing.T) { + cli.runTestsOnNewDB(t, nil, "prepared_timestamp", func(dbt *testkit.DBTestKit) { + dbt.MustExec("create table test (a timestamp, b time)") + dbt.MustExec("set time_zone='+00:00'") + insertStmt := dbt.MustPrepare("insert test values (?, ?)") defer insertStmt.Close() vts := time.Unix(1, 1) vt := time.Unix(-1, 1) - dbt.mustExecPrepared(insertStmt, vts, vt) - selectStmt := dbt.mustPrepare("select * from test where a = ? and b = ?") + dbt.MustExecPrepared(insertStmt, vts, vt) + selectStmt := dbt.MustPrepare("select * from test where a = ? and b = ?") defer selectStmt.Close() - rows := dbt.mustQueryPrepared(selectStmt, vts, vt) - t.Assert(rows.Next(), IsTrue) + rows := dbt.MustQueryPrepared(selectStmt, vts, vt) + require.True(t, rows.Next()) var outA, outB string err := rows.Scan(&outA, &outB) - t.Assert(err, IsNil) - t.Assert(outA, Equals, "1970-01-01 00:00:01") - t.Assert(outB, Equals, "23:59:59") + require.NoError(t, err) + require.Equal(t, "1970-01-01 00:00:01", outA) + require.Equal(t, "23:59:59", outB) }) } -func (cli *testServerClient) runTestLoadDataWithSelectIntoOutfile(c *C, server *Server) { - cli.runTestsOnNewDB(c, func(config *mysql.Config) { +func (cli *testServerClient) runTestLoadDataWithSelectIntoOutfile(t *testing.T, server *Server) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params["sql_mode"] = "''" - }, "SelectIntoOutfile", func(dbt *DBTest) { - dbt.mustExec("create table t (i int, r real, d decimal(10, 5), s varchar(100), dt datetime, ts timestamp, j json)") - dbt.mustExec("insert into t values (1, 1.1, 0.1, 'a', '2000-01-01', '01:01:01', '[1]')") - dbt.mustExec("insert into t values (2, 2.2, 0.2, 'b', '2000-02-02', '02:02:02', '[1,2]')") - dbt.mustExec("insert into t values (null, null, null, null, '2000-03-03', '03:03:03', '[1,2,3]')") - dbt.mustExec("insert into t values (4, 4.4, 0.4, 'd', null, null, null)") + }, "SelectIntoOutfile", func(dbt *testkit.DBTestKit) { + dbt.MustExec("create table t (i int, r real, d decimal(10, 5), s varchar(100), dt datetime, ts timestamp, j json)") + dbt.MustExec("insert into t values (1, 1.1, 0.1, 'a', '2000-01-01', '01:01:01', '[1]')") + dbt.MustExec("insert into t values (2, 2.2, 0.2, 'b', '2000-02-02', '02:02:02', '[1,2]')") + dbt.MustExec("insert into t values (null, null, null, null, '2000-03-03', '03:03:03', '[1,2,3]')") + dbt.MustExec("insert into t values (4, 4.4, 0.4, 'd', null, null, null)") outfile := filepath.Join(os.TempDir(), fmt.Sprintf("select_into_outfile_%v_%d.csv", time.Now().UnixNano(), rand.Int())) // On windows use fmt.Sprintf("%q") to escape \ for SQL, // outfile may be 'C:\Users\genius\AppData\Local\Temp\select_into_outfile_1582732846769492000_8074605509026837941.csv' // Without quote, after SQL escape it would become: // 'C:UsersgeniusAppDataLocalTempselect_into_outfile_1582732846769492000_8074605509026837941.csv' - dbt.mustExec(fmt.Sprintf("select * from t into outfile %q", outfile)) + dbt.MustExec(fmt.Sprintf("select * from t into outfile %q", outfile)) defer func() { - c.Assert(os.Remove(outfile), IsNil) + require.NoError(t, os.Remove(outfile)) }() - dbt.mustExec("create table t1 (i int, r real, d decimal(10, 5), s varchar(100), dt datetime, ts timestamp, j json)") - dbt.mustExec(fmt.Sprintf("load data local infile %q into table t1", outfile)) + dbt.MustExec("create table t1 (i int, r real, d decimal(10, 5), s varchar(100), dt datetime, ts timestamp, j json)") + dbt.MustExec(fmt.Sprintf("load data local infile %q into table t1", outfile)) fetchResults := func(table string) [][]interface{} { var res [][]interface{} - row := dbt.mustQuery("select * from " + table + " order by i") + row := dbt.MustQuery("select * from " + table + " order by i") for row.Next() { r := make([]interface{}, 7) - c.Assert(row.Scan(&r[0], &r[1], &r[2], &r[3], &r[4], &r[5], &r[6]), IsNil) + require.NoError(t, row.Scan(&r[0], &r[1], &r[2], &r[3], &r[4], &r[5], &r[6])) res = append(res, r) } - c.Assert(row.Close(), IsNil) + require.NoError(t, row.Close()) return res } res := fetchResults("t") res1 := fetchResults("t1") - c.Assert(len(res), Equals, len(res1)) + require.Equal(t, len(res1), len(res)) for i := range res { for j := range res[i] { // using Sprintf to avoid some uncomparable types - c.Assert(fmt.Sprintf("%v", res[i][j]), Equals, fmt.Sprintf("%v", res1[i][j])) + require.Equal(t, fmt.Sprintf("%v", res1[i][j]), fmt.Sprintf("%v", res[i][j])) } } }) } -func (cli *testServerClient) runTestLoadDataForSlowLog(c *C, server *Server) { + +func (cli *testServerClient) runTestLoadDataForSlowLog(t *testing.T, server *Server) { path := "/tmp/load_data_test.csv" fp, err := os.OpenFile(path, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) - c.Assert(err, IsNil) - c.Assert(fp, NotNil) + require.NoError(t, err) + require.NotNil(t, fp) defer func() { err = fp.Close() - c.Assert(err, IsNil) + require.NoError(t, err) err = os.Remove(path) - c.Assert(err, IsNil) + require.NoError(t, err) }() _, err = fp.WriteString( "1 1\n" + @@ -452,69 +409,69 @@ func (cli *testServerClient) runTestLoadDataForSlowLog(c *C, server *Server) { "3 3\n" + "4 4\n" + "5 5\n") - c.Assert(err, IsNil) + require.NoError(t, err) - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params["sql_mode"] = "''" - }, "load_data_slow_query", func(dbt *DBTest) { - dbt.mustExec("create table t_slow (a int key, b int)") + }, "load_data_slow_query", func(dbt *testkit.DBTestKit) { + dbt.MustExec("create table t_slow (a int key, b int)") defer func() { - dbt.mustExec("set tidb_slow_log_threshold=300;") - dbt.mustExec("set @@global.tidb_enable_stmt_summary=0") + dbt.MustExec("set tidb_slow_log_threshold=300;") + dbt.MustExec("set @@global.tidb_enable_stmt_summary=0") }() - dbt.mustExec("set tidb_slow_log_threshold=0;") - dbt.mustExec("set @@global.tidb_enable_stmt_summary=1") + dbt.MustExec("set tidb_slow_log_threshold=0;") + dbt.MustExec("set @@global.tidb_enable_stmt_summary=1") query := fmt.Sprintf("load data local infile %q into table t_slow", path) - dbt.mustExec(query) - dbt.mustExec("insert ignore into t_slow values (1,1);") + dbt.MustExec(query) + dbt.MustExec("insert ignore into t_slow values (1,1);") checkPlan := func(rows *sql.Rows, expectPlan string) { - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.Truef(t, rows.Next(), "unexpected data") var plan sql.NullString err = rows.Scan(&plan) - dbt.Check(err, IsNil) + require.NoError(t, err) planStr := strings.ReplaceAll(plan.String, "\t", " ") planStr = strings.ReplaceAll(planStr, "\n", " ") - c.Assert(planStr, Matches, expectPlan) + require.Regexp(t, expectPlan, planStr) } // Test for record slow log for load data statement. - rows := dbt.mustQuery("select plan from information_schema.slow_query where query like 'load data local infile % into table t_slow;' order by time desc limit 1") + rows := dbt.MustQuery("select plan from information_schema.slow_query where query like 'load data local infile % into table t_slow;' order by time desc limit 1") expectedPlan := ".*LoadData.* time.* loops.* prepare.* check_insert.* mem_insert_time:.* prefetch.* rpc.* commit_txn.*" checkPlan(rows, expectedPlan) // Test for record statements_summary for load data statement. - rows = dbt.mustQuery("select plan from information_schema.STATEMENTS_SUMMARY where QUERY_SAMPLE_TEXT like 'load data local infile %' limit 1") + rows = dbt.MustQuery("select plan from information_schema.STATEMENTS_SUMMARY where QUERY_SAMPLE_TEXT like 'load data local infile %' limit 1") checkPlan(rows, expectedPlan) // Test log normal statement after executing load date. - rows = dbt.mustQuery("select plan from information_schema.slow_query where query = 'insert ignore into t_slow values (1,1);' order by time desc limit 1") + rows = dbt.MustQuery("select plan from information_schema.slow_query where query = 'insert ignore into t_slow values (1,1);' order by time desc limit 1") expectedPlan = ".*Insert.* time.* loops.* prepare.* check_insert.* mem_insert_time:.* prefetch.* rpc.*" checkPlan(rows, expectedPlan) }) } -func (cli *testServerClient) prepareLoadDataFile(c *C, path string, rows ...string) { +func (cli *testServerClient) prepareLoadDataFile(t *testing.T, path string, rows ...string) { fp, err := os.OpenFile(path, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) - c.Assert(err, IsNil) - c.Assert(fp, NotNil) + require.NoError(t, err) + require.NotNil(t, fp) defer func() { err = fp.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() for _, row := range rows { fields := strings.Split(row, " ") _, err = fp.WriteString(strings.Join(fields, "\t")) _, err = fp.WriteString("\n") } - c.Assert(err, IsNil) + require.NoError(t, err) } -func (cli *testServerClient) runTestLoadDataAutoRandom(c *C) { +func (cli *testServerClient) runTestLoadDataAutoRandom(t *testing.T) { path := "/tmp/load_data_txn_error.csv" fp, err := os.OpenFile(path, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) - c.Assert(err, IsNil) - c.Assert(fp, NotNil) + require.NoError(t, err) + require.NotNil(t, fp) defer func() { _ = os.Remove(path) @@ -529,9 +486,9 @@ func (cli *testServerClient) runTestLoadDataAutoRandom(c *C) { str2 := strconv.Itoa(n2) row := str1 + "\t" + str2 _, err := fp.WriteString(row) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = fp.WriteString("\n") - c.Assert(err, IsNil) + require.NoError(t, err) if i == 0 { cksum1 = n1 @@ -543,33 +500,33 @@ func (cli *testServerClient) runTestLoadDataAutoRandom(c *C) { } err = fp.Close() - c.Assert(err, IsNil) + require.NoError(t, err) - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params["sql_mode"] = "''" - }, "load_data_batch_dml", func(dbt *DBTest) { + }, "load_data_batch_dml", func(dbt *testkit.DBTestKit) { // Set batch size, and check if load data got a invalid txn error. - dbt.mustExec("set @@session.tidb_dml_batch_size = 128") - dbt.mustExec("drop table if exists t") - dbt.mustExec("create table t(c1 bigint auto_random primary key, c2 bigint, c3 bigint)") - dbt.mustExec(fmt.Sprintf("load data local infile %q into table t (c2, c3)", path)) - rows := dbt.mustQuery("select count(*) from t") - cli.checkRows(c, rows, "50000") - rows = dbt.mustQuery("select bit_xor(c2), bit_xor(c3) from t") + dbt.MustExec("set @@session.tidb_dml_batch_size = 128") + dbt.MustExec("drop table if exists t") + dbt.MustExec("create table t(c1 bigint auto_random primary key, c2 bigint, c3 bigint)") + dbt.MustExec(fmt.Sprintf("load data local infile %q into table t (c2, c3)", path)) + rows := dbt.MustQuery("select count(*) from t") + cli.checkRows(t, rows, "50000") + rows = dbt.MustQuery("select bit_xor(c2), bit_xor(c3) from t") res := strconv.Itoa(cksum1) res = res + " " res = res + strconv.Itoa(cksum2) - cli.checkRows(c, rows, res) + cli.checkRows(t, rows, res) }) } -func (cli *testServerClient) runTestLoadDataAutoRandomWithSpecialTerm(c *C) { +func (cli *testServerClient) runTestLoadDataAutoRandomWithSpecialTerm(t *testing.T) { path := "/tmp/load_data_txn_error_term.csv" fp, err := os.OpenFile(path, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) - c.Assert(err, IsNil) - c.Assert(fp, NotNil) + require.NoError(t, err) + require.NotNil(t, fp) defer func() { _ = os.Remove(path) @@ -584,11 +541,11 @@ func (cli *testServerClient) runTestLoadDataAutoRandomWithSpecialTerm(c *C) { str2 := strconv.Itoa(n2) row := "'" + str1 + "','" + str2 + "'" _, err := fp.WriteString(row) - c.Assert(err, IsNil) + require.NoError(t, err) if i != 49999 { _, err = fp.WriteString("|") } - c.Assert(err, IsNil) + require.NoError(t, err) if i == 0 { cksum1 = n1 @@ -600,39 +557,39 @@ func (cli *testServerClient) runTestLoadDataAutoRandomWithSpecialTerm(c *C) { } err = fp.Close() - c.Assert(err, IsNil) + require.NoError(t, err) - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params = map[string]string{"sql_mode": "''"} - }, "load_data_batch_dml", func(dbt *DBTest) { + }, "load_data_batch_dml", func(dbt *testkit.DBTestKit) { // Set batch size, and check if load data got a invalid txn error. - dbt.mustExec("set @@session.tidb_dml_batch_size = 128") - dbt.mustExec("drop table if exists t1") - dbt.mustExec("create table t1(c1 bigint auto_random primary key, c2 bigint, c3 bigint)") - dbt.mustExec(fmt.Sprintf("load data local infile %q into table t1 fields terminated by ',' enclosed by '\\'' lines terminated by '|' (c2, c3)", path)) - rows := dbt.mustQuery("select count(*) from t1") - cli.checkRows(c, rows, "50000") - rows = dbt.mustQuery("select bit_xor(c2), bit_xor(c3) from t1") + dbt.MustExec("set @@session.tidb_dml_batch_size = 128") + dbt.MustExec("drop table if exists t1") + dbt.MustExec("create table t1(c1 bigint auto_random primary key, c2 bigint, c3 bigint)") + dbt.MustExec(fmt.Sprintf("load data local infile %q into table t1 fields terminated by ',' enclosed by '\\'' lines terminated by '|' (c2, c3)", path)) + rows := dbt.MustQuery("select count(*) from t1") + cli.checkRows(t, rows, "50000") + rows = dbt.MustQuery("select bit_xor(c2), bit_xor(c3) from t1") res := strconv.Itoa(cksum1) res = res + " " res = res + strconv.Itoa(cksum2) - cli.checkRows(c, rows, res) + cli.checkRows(t, rows, res) }) } -func (cli *testServerClient) runTestLoadDataForListPartition(c *C) { +func (cli *testServerClient) runTestLoadDataForListPartition(t *testing.T) { path := "/tmp/load_data_list_partition.csv" defer func() { _ = os.Remove(path) }() - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params["sql_mode"] = "''" - }, "load_data_list_partition", func(dbt *DBTest) { - dbt.mustExec("set @@session.tidb_enable_list_partition = ON") - dbt.mustExec(`create table t (id int, name varchar(10), + }, "load_data_list_partition", func(dbt *testkit.DBTestKit) { + dbt.MustExec("set @@session.tidb_enable_list_partition = ON") + dbt.MustExec(`create table t (id int, name varchar(10), unique index idx (id)) partition by list (id) ( partition p0 values in (3,5,6,9,17), partition p1 values in (1,2,10,11,19,20), @@ -640,48 +597,48 @@ func (cli *testServerClient) runTestLoadDataForListPartition(c *C) { partition p3 values in (7,8,15,16,null) );`) // Test load data into 1 partition. - cli.prepareLoadDataFile(c, path, "1 a", "2 b") - dbt.mustExec(fmt.Sprintf("load data local infile %q into table t", path)) - rows := dbt.mustQuery("select * from t partition(p1) order by id") - cli.checkRows(c, rows, "1 a", "2 b") + cli.prepareLoadDataFile(t, path, "1 a", "2 b") + dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) + rows := dbt.MustQuery("select * from t partition(p1) order by id") + cli.checkRows(t, rows, "1 a", "2 b") // Test load data into multi-partitions. - dbt.mustExec("delete from t") - cli.prepareLoadDataFile(c, path, "1 a", "3 c", "4 e") - dbt.mustExec(fmt.Sprintf("load data local infile %q into table t", path)) - rows = dbt.mustQuery("select * from t order by id") - cli.checkRows(c, rows, "1 a", "3 c", "4 e") + dbt.MustExec("delete from t") + cli.prepareLoadDataFile(t, path, "1 a", "3 c", "4 e") + dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) + rows = dbt.MustQuery("select * from t order by id") + cli.checkRows(t, rows, "1 a", "3 c", "4 e") // Test load data meet duplicate error. - cli.prepareLoadDataFile(c, path, "1 x", "2 b", "2 x", "7 a") - dbt.mustExec(fmt.Sprintf("load data local infile %q into table t", path)) - rows = dbt.mustQuery("show warnings") - cli.checkRows(c, rows, + cli.prepareLoadDataFile(t, path, "1 x", "2 b", "2 x", "7 a") + dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) + rows = dbt.MustQuery("show warnings") + cli.checkRows(t, rows, "Warning 1062 Duplicate entry '1' for key 'idx'", "Warning 1062 Duplicate entry '2' for key 'idx'") - rows = dbt.mustQuery("select * from t order by id") - cli.checkRows(c, rows, "1 a", "2 b", "3 c", "4 e", "7 a") + rows = dbt.MustQuery("select * from t order by id") + cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "7 a") // Test load data meet no partition warning. - cli.prepareLoadDataFile(c, path, "5 a", "100 x") - _, err := dbt.db.Exec(fmt.Sprintf("load data local infile %q into table t", path)) - c.Assert(err, IsNil) - rows = dbt.mustQuery("show warnings") - cli.checkRows(c, rows, "Warning 1526 Table has no partition for value 100") - rows = dbt.mustQuery("select * from t order by id") - cli.checkRows(c, rows, "1 a", "2 b", "3 c", "4 e", "5 a", "7 a") + cli.prepareLoadDataFile(t, path, "5 a", "100 x") + _, err := dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t", path)) + require.NoError(t, err) + rows = dbt.MustQuery("show warnings") + cli.checkRows(t, rows, "Warning 1526 Table has no partition for value 100") + rows = dbt.MustQuery("select * from t order by id") + cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "5 a", "7 a") }) } -func (cli *testServerClient) runTestLoadDataForListPartition2(c *C) { +func (cli *testServerClient) runTestLoadDataForListPartition2(t *testing.T) { path := "/tmp/load_data_list_partition.csv" defer func() { _ = os.Remove(path) }() - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params["sql_mode"] = "''" - }, "load_data_list_partition", func(dbt *DBTest) { - dbt.mustExec("set @@session.tidb_enable_list_partition = ON") - dbt.mustExec(`create table t (id int, name varchar(10),b int generated always as (length(name)+1) virtual, + }, "load_data_list_partition", func(dbt *testkit.DBTestKit) { + dbt.MustExec("set @@session.tidb_enable_list_partition = ON") + dbt.MustExec(`create table t (id int, name varchar(10),b int generated always as (length(name)+1) virtual, unique index idx (id,b)) partition by list (id*2 + b*b + b*b - b*b*2 - abs(id)) ( partition p0 values in (3,5,6,9,17), partition p1 values in (1,2,10,11,19,20), @@ -689,48 +646,48 @@ func (cli *testServerClient) runTestLoadDataForListPartition2(c *C) { partition p3 values in (7,8,15,16,null) );`) // Test load data into 1 partition. - cli.prepareLoadDataFile(c, path, "1 a", "2 b") - dbt.mustExec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) - rows := dbt.mustQuery("select id,name from t partition(p1) order by id") - cli.checkRows(c, rows, "1 a", "2 b") + cli.prepareLoadDataFile(t, path, "1 a", "2 b") + dbt.MustExec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) + rows := dbt.MustQuery("select id,name from t partition(p1) order by id") + cli.checkRows(t, rows, "1 a", "2 b") // Test load data into multi-partitions. - dbt.mustExec("delete from t") - cli.prepareLoadDataFile(c, path, "1 a", "3 c", "4 e") - dbt.mustExec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) - rows = dbt.mustQuery("select id,name from t order by id") - cli.checkRows(c, rows, "1 a", "3 c", "4 e") + dbt.MustExec("delete from t") + cli.prepareLoadDataFile(t, path, "1 a", "3 c", "4 e") + dbt.MustExec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) + rows = dbt.MustQuery("select id,name from t order by id") + cli.checkRows(t, rows, "1 a", "3 c", "4 e") // Test load data meet duplicate error. - cli.prepareLoadDataFile(c, path, "1 x", "2 b", "2 x", "7 a") - dbt.mustExec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) - rows = dbt.mustQuery("show warnings") - cli.checkRows(c, rows, + cli.prepareLoadDataFile(t, path, "1 x", "2 b", "2 x", "7 a") + dbt.MustExec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) + rows = dbt.MustQuery("show warnings") + cli.checkRows(t, rows, "Warning 1062 Duplicate entry '1-2' for key 'idx'", "Warning 1062 Duplicate entry '2-2' for key 'idx'") - rows = dbt.mustQuery("select id,name from t order by id") - cli.checkRows(c, rows, "1 a", "2 b", "3 c", "4 e", "7 a") + rows = dbt.MustQuery("select id,name from t order by id") + cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "7 a") // Test load data meet no partition warning. - cli.prepareLoadDataFile(c, path, "5 a", "100 x") - _, err := dbt.db.Exec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) - c.Assert(err, IsNil) - rows = dbt.mustQuery("show warnings") - cli.checkRows(c, rows, "Warning 1526 Table has no partition for value 100") - rows = dbt.mustQuery("select id,name from t order by id") - cli.checkRows(c, rows, "1 a", "2 b", "3 c", "4 e", "5 a", "7 a") + cli.prepareLoadDataFile(t, path, "5 a", "100 x") + _, err := dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) + require.NoError(t, err) + rows = dbt.MustQuery("show warnings") + cli.checkRows(t, rows, "Warning 1526 Table has no partition for value 100") + rows = dbt.MustQuery("select id,name from t order by id") + cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "5 a", "7 a") }) } -func (cli *testServerClient) runTestLoadDataForListColumnPartition(c *C) { +func (cli *testServerClient) runTestLoadDataForListColumnPartition(t *testing.T) { path := "/tmp/load_data_list_partition.csv" defer func() { _ = os.Remove(path) }() - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params["sql_mode"] = "''" - }, "load_data_list_partition", func(dbt *DBTest) { - dbt.mustExec("set @@session.tidb_enable_list_partition = ON") - dbt.mustExec(`create table t (id int, name varchar(10), + }, "load_data_list_partition", func(dbt *testkit.DBTestKit) { + dbt.MustExec("set @@session.tidb_enable_list_partition = ON") + dbt.MustExec(`create table t (id int, name varchar(10), unique index idx (id)) partition by list columns (id) ( partition p0 values in (3,5,6,9,17), partition p1 values in (1,2,10,11,19,20), @@ -738,96 +695,96 @@ func (cli *testServerClient) runTestLoadDataForListColumnPartition(c *C) { partition p3 values in (7,8,15,16,null) );`) // Test load data into 1 partition. - cli.prepareLoadDataFile(c, path, "1 a", "2 b") - dbt.mustExec(fmt.Sprintf("load data local infile %q into table t", path)) - rows := dbt.mustQuery("select * from t partition(p1) order by id") - cli.checkRows(c, rows, "1 a", "2 b") + cli.prepareLoadDataFile(t, path, "1 a", "2 b") + dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) + rows := dbt.MustQuery("select * from t partition(p1) order by id") + cli.checkRows(t, rows, "1 a", "2 b") // Test load data into multi-partitions. - dbt.mustExec("delete from t") - cli.prepareLoadDataFile(c, path, "1 a", "3 c", "4 e") - dbt.mustExec(fmt.Sprintf("load data local infile %q into table t", path)) - rows = dbt.mustQuery("select * from t order by id") - cli.checkRows(c, rows, "1 a", "3 c", "4 e") + dbt.MustExec("delete from t") + cli.prepareLoadDataFile(t, path, "1 a", "3 c", "4 e") + dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) + rows = dbt.MustQuery("select * from t order by id") + cli.checkRows(t, rows, "1 a", "3 c", "4 e") // Test load data meet duplicate error. - cli.prepareLoadDataFile(c, path, "1 x", "2 b", "2 x", "7 a") - dbt.mustExec(fmt.Sprintf("load data local infile %q into table t", path)) - rows = dbt.mustQuery("show warnings") - cli.checkRows(c, rows, + cli.prepareLoadDataFile(t, path, "1 x", "2 b", "2 x", "7 a") + dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) + rows = dbt.MustQuery("show warnings") + cli.checkRows(t, rows, "Warning 1062 Duplicate entry '1' for key 'idx'", "Warning 1062 Duplicate entry '2' for key 'idx'") - rows = dbt.mustQuery("select * from t order by id") - cli.checkRows(c, rows, "1 a", "2 b", "3 c", "4 e", "7 a") + rows = dbt.MustQuery("select * from t order by id") + cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "7 a") // Test load data meet no partition warning. - cli.prepareLoadDataFile(c, path, "5 a", "100 x") - _, err := dbt.db.Exec(fmt.Sprintf("load data local infile %q into table t", path)) - c.Assert(err, IsNil) - rows = dbt.mustQuery("show warnings") - cli.checkRows(c, rows, "Warning 1526 Table has no partition for value from column_list") - rows = dbt.mustQuery("select id,name from t order by id") - cli.checkRows(c, rows, "1 a", "2 b", "3 c", "4 e", "5 a", "7 a") + cli.prepareLoadDataFile(t, path, "5 a", "100 x") + _, err := dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t", path)) + require.NoError(t, err) + rows = dbt.MustQuery("show warnings") + cli.checkRows(t, rows, "Warning 1526 Table has no partition for value from column_list") + rows = dbt.MustQuery("select id,name from t order by id") + cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "5 a", "7 a") }) } -func (cli *testServerClient) runTestLoadDataForListColumnPartition2(c *C) { +func (cli *testServerClient) runTestLoadDataForListColumnPartition2(t *testing.T) { path := "/tmp/load_data_list_partition.csv" defer func() { _ = os.Remove(path) }() - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params["sql_mode"] = "''" - }, "load_data_list_partition", func(dbt *DBTest) { - dbt.mustExec("set @@session.tidb_enable_list_partition = ON") - dbt.mustExec(`create table t (location varchar(10), id int, a int, unique index idx (location,id)) partition by list columns (location,id) ( + }, "load_data_list_partition", func(dbt *testkit.DBTestKit) { + dbt.MustExec("set @@session.tidb_enable_list_partition = ON") + dbt.MustExec(`create table t (location varchar(10), id int, a int, unique index idx (location,id)) partition by list columns (location,id) ( partition p_west values in (('w', 1),('w', 2),('w', 3),('w', 4)), partition p_east values in (('e', 5),('e', 6),('e', 7),('e', 8)), partition p_north values in (('n', 9),('n',10),('n',11),('n',12)), partition p_south values in (('s',13),('s',14),('s',15),('s',16)) );`) // Test load data into 1 partition. - cli.prepareLoadDataFile(c, path, "w 1 1", "w 2 2") - dbt.mustExec(fmt.Sprintf("load data local infile %q into table t", path)) - rows := dbt.mustQuery("select * from t partition(p_west) order by id") - cli.checkRows(c, rows, "w 1 1", "w 2 2") + cli.prepareLoadDataFile(t, path, "w 1 1", "w 2 2") + dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) + rows := dbt.MustQuery("select * from t partition(p_west) order by id") + cli.checkRows(t, rows, "w 1 1", "w 2 2") // Test load data into multi-partitions. - dbt.mustExec("delete from t") - cli.prepareLoadDataFile(c, path, "w 1 1", "e 5 5", "n 9 9") - dbt.mustExec(fmt.Sprintf("load data local infile %q into table t", path)) - rows = dbt.mustQuery("select * from t order by id") - cli.checkRows(c, rows, "w 1 1", "e 5 5", "n 9 9") + dbt.MustExec("delete from t") + cli.prepareLoadDataFile(t, path, "w 1 1", "e 5 5", "n 9 9") + dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) + rows = dbt.MustQuery("select * from t order by id") + cli.checkRows(t, rows, "w 1 1", "e 5 5", "n 9 9") // Test load data meet duplicate error. - cli.prepareLoadDataFile(c, path, "w 1 2", "w 2 2") - _, err := dbt.db.Exec(fmt.Sprintf("load data local infile %q into table t", path)) - c.Assert(err, IsNil) - rows = dbt.mustQuery("show warnings") - cli.checkRows(c, rows, "Warning 1062 Duplicate entry 'w-1' for key 'idx'") - rows = dbt.mustQuery("select * from t order by id") - cli.checkRows(c, rows, "w 1 1", "w 2 2", "e 5 5", "n 9 9") + cli.prepareLoadDataFile(t, path, "w 1 2", "w 2 2") + _, err := dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t", path)) + require.NoError(t, err) + rows = dbt.MustQuery("show warnings") + cli.checkRows(t, rows, "Warning 1062 Duplicate entry 'w-1' for key 'idx'") + rows = dbt.MustQuery("select * from t order by id") + cli.checkRows(t, rows, "w 1 1", "w 2 2", "e 5 5", "n 9 9") // Test load data meet no partition warning. - cli.prepareLoadDataFile(c, path, "w 3 3", "w 5 5", "e 8 8") - _, err = dbt.db.Exec(fmt.Sprintf("load data local infile %q into table t", path)) - c.Assert(err, IsNil) - rows = dbt.mustQuery("show warnings") - cli.checkRows(c, rows, "Warning 1526 Table has no partition for value from column_list") - cli.prepareLoadDataFile(c, path, "x 1 1", "w 1 1") - _, err = dbt.db.Exec(fmt.Sprintf("load data local infile %q into table t", path)) - c.Assert(err, IsNil) - rows = dbt.mustQuery("show warnings") - cli.checkRows(c, rows, + cli.prepareLoadDataFile(t, path, "w 3 3", "w 5 5", "e 8 8") + _, err = dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t", path)) + require.NoError(t, err) + rows = dbt.MustQuery("show warnings") + cli.checkRows(t, rows, "Warning 1526 Table has no partition for value from column_list") + cli.prepareLoadDataFile(t, path, "x 1 1", "w 1 1") + _, err = dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t", path)) + require.NoError(t, err) + rows = dbt.MustQuery("show warnings") + cli.checkRows(t, rows, "Warning 1526 Table has no partition for value from column_list", "Warning 1062 Duplicate entry 'w-1' for key 'idx'") - rows = dbt.mustQuery("select * from t order by id") - cli.checkRows(c, rows, "w 1 1", "w 2 2", "w 3 3", "e 5 5", "e 8 8", "n 9 9") + rows = dbt.MustQuery("select * from t order by id") + cli.checkRows(t, rows, "w 1 1", "w 2 2", "w 3 3", "e 5 5", "e 8 8", "n 9 9") }) } -func (cli *testServerClient) checkRows(c *C, rows *sql.Rows, expectedRows ...string) { +func (cli *testServerClient) checkRows(t *testing.T, rows *sql.Rows, expectedRows ...string) { buf := bytes.NewBuffer(nil) result := make([]string, 0, 2) for rows.Next() { cols, err := rows.Columns() - c.Assert(err, IsNil) + require.NoError(t, err) rawResult := make([][]byte, len(cols)) dest := make([]interface{}, len(cols)) for i := range rawResult { @@ -835,7 +792,7 @@ func (cli *testServerClient) checkRows(c *C, rows *sql.Rows, expectedRows ...str } err = rows.Scan(dest...) - c.Assert(err, IsNil) + require.NoError(t, err) buf.Reset() for i, raw := range rawResult { if i > 0 { @@ -849,20 +806,21 @@ func (cli *testServerClient) checkRows(c *C, rows *sql.Rows, expectedRows ...str } result = append(result, buf.String()) } - c.Assert(strings.Join(result, "\n"), Equals, strings.Join(expectedRows, "\n")) + + require.Equal(t, strings.Join(expectedRows, "\n"), strings.Join(result, "\n")) } -func (cli *testServerClient) runTestLoadData(c *C, server *Server) { +func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { // create a file and write data. path := "/tmp/load_data_test.csv" fp, err := os.Create(path) - c.Assert(err, IsNil) - c.Assert(fp, NotNil) + require.NoError(t, err) + require.NotNil(t, fp) defer func() { err = fp.Close() - c.Assert(err, IsNil) + require.NoError(t, err) err = os.Remove(path) - c.Assert(err, IsNil) + require.NoError(t, err) }() _, err = fp.WriteString("\n" + "xxx row1_col1 - row1_col2 1abc\n" + @@ -870,7 +828,7 @@ func (cli *testServerClient) runTestLoadData(c *C, server *Server) { "xxxy row3_col1 - row3_col2 \n" + "xxx row4_col1 - 900\n" + "xxx row5_col1 - row5_col3") - c.Assert(err, IsNil) + require.NoError(t, err) originalTxnTotalSizeLimit := kv.TxnTotalSizeLimit // If the MemBuffer can't be committed once in each batch, it will return an error like "transaction is too large". @@ -878,650 +836,650 @@ func (cli *testServerClient) runTestLoadData(c *C, server *Server) { defer func() { kv.TxnTotalSizeLimit = originalTxnTotalSizeLimit }() // support ClientLocalFiles capability - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params["sql_mode"] = "''" - }, "LoadData", func(dbt *DBTest) { - dbt.mustExec("set @@tidb_dml_batch_size = 3") - dbt.mustExec("create table test (a varchar(255), b varchar(255) default 'default value', c int not null auto_increment, primary key(c))") - dbt.mustExec("create view v1 as select 1") - dbt.mustExec("create sequence s1") + }, "LoadData", func(dbt *testkit.DBTestKit) { + dbt.MustExec("set @@tidb_dml_batch_size = 3") + dbt.MustExec("create table test (a varchar(255), b varchar(255) default 'default value', c int not null auto_increment, primary key(c))") + dbt.MustExec("create view v1 as select 1") + dbt.MustExec("create sequence s1") // can't insert into views (in TiDB) or sequences. issue #20880 - _, err = dbt.db.Exec("load data local infile '/tmp/load_data_test.csv' into table v1") - dbt.Assert(err, NotNil) - dbt.Assert(err.Error(), Equals, "Error 1105: can only load data into base tables") - _, err = dbt.db.Exec("load data local infile '/tmp/load_data_test.csv' into table s1") - dbt.Assert(err, NotNil) - dbt.Assert(err.Error(), Equals, "Error 1105: can only load data into base tables") - - rs, err1 := dbt.db.Exec("load data local infile '/tmp/load_data_test.csv' into table test") - dbt.Assert(err1, IsNil) + _, err = dbt.GetDB().Exec("load data local infile '/tmp/load_data_test.csv' into table v1") + require.Error(t, err) + require.Equal(t, "Error 1105: can only load data into base tables", err.Error()) + _, err = dbt.GetDB().Exec("load data local infile '/tmp/load_data_test.csv' into table s1") + require.Error(t, err) + require.Equal(t, "Error 1105: can only load data into base tables", err.Error()) + + rs, err1 := dbt.GetDB().Exec("load data local infile '/tmp/load_data_test.csv' into table test") + require.NoError(t, err1) lastID, err1 := rs.LastInsertId() - dbt.Assert(err1, IsNil) - dbt.Assert(lastID, Equals, int64(1)) + require.NoError(t, err1) + require.Equal(t, int64(1), lastID) affectedRows, err1 := rs.RowsAffected() - dbt.Assert(err1, IsNil) - dbt.Assert(affectedRows, Equals, int64(5)) + require.NoError(t, err1) + require.Equal(t, int64(5), affectedRows) var ( a string b string bb sql.NullString cc int ) - rows := dbt.mustQuery("select * from test") - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + rows := dbt.MustQuery("select * from test") + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &bb, &cc) - dbt.Check(err, IsNil) - dbt.Check(a, DeepEquals, "") - dbt.Check(bb.String, DeepEquals, "") - dbt.Check(cc, DeepEquals, 1) - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Empty(t, a) + require.Empty(t, bb.String) + require.Equal(t, 1, cc) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &cc) - c.Assert(err, IsNil) - dbt.Check(a, DeepEquals, "xxx row2_col1") - dbt.Check(b, DeepEquals, "- row2_col2") - dbt.Check(cc, DeepEquals, 2) - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, "xxx row2_col1", a) + require.Equal(t, "- row2_col2", b) + require.Equal(t, 2, cc) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &cc) - c.Assert(err, IsNil) - dbt.Check(a, DeepEquals, "xxxy row3_col1") - dbt.Check(b, DeepEquals, "- row3_col2") - dbt.Check(cc, DeepEquals, 3) - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, "xxxy row3_col1", a) + require.Equal(t, "- row3_col2", b) + require.Equal(t, 3, cc) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &cc) - c.Assert(err, IsNil) - dbt.Check(a, DeepEquals, "xxx row4_col1") - dbt.Check(b, DeepEquals, "- ") - dbt.Check(cc, DeepEquals, 4) - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, "xxx row4_col1", a) + require.Equal(t, "- ", b) + require.Equal(t, 4, cc) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &cc) - c.Assert(err, IsNil) - dbt.Check(a, DeepEquals, "xxx row5_col1") - dbt.Check(b, DeepEquals, "- ") - dbt.Check(cc, DeepEquals, 5) - dbt.Check(rows.Next(), IsFalse, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, "xxx row5_col1", a) + require.Equal(t, "- ", b) + require.Equal(t, 5, cc) + require.Falsef(t, rows.Next(), "unexpected data") rows.Close() // specify faileds and lines - dbt.mustExec("delete from test") - dbt.mustExec("set @@tidb_dml_batch_size = 3") - rs, err = dbt.db.Exec("load data local infile '/tmp/load_data_test.csv' into table test fields terminated by '\t- ' lines starting by 'xxx ' terminated by '\n'") - dbt.Assert(err, IsNil) + dbt.MustExec("delete from test") + dbt.MustExec("set @@tidb_dml_batch_size = 3") + rs, err = dbt.GetDB().Exec("load data local infile '/tmp/load_data_test.csv' into table test fields terminated by '\t- ' lines starting by 'xxx ' terminated by '\n'") + require.NoError(t, err) lastID, err = rs.LastInsertId() - dbt.Assert(err, IsNil) - dbt.Assert(lastID, Equals, int64(6)) + require.NoError(t, err) + require.Equal(t, int64(6), lastID) affectedRows, err = rs.RowsAffected() - dbt.Assert(err, IsNil) - dbt.Assert(affectedRows, Equals, int64(4)) - rows = dbt.mustQuery("select * from test") - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, int64(4), affectedRows) + rows = dbt.MustQuery("select * from test") + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &cc) - c.Assert(err, IsNil) - dbt.Check(a, DeepEquals, "row1_col1") - dbt.Check(b, DeepEquals, "row1_col2\t1abc") - dbt.Check(cc, DeepEquals, 6) - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, "row1_col1", a) + require.Equal(t, "row1_col2\t1abc", b) + require.Equal(t, 6, cc) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &cc) - c.Assert(err, IsNil) - dbt.Check(a, DeepEquals, "row2_col1") - dbt.Check(b, DeepEquals, "row2_col2\t") - dbt.Check(cc, DeepEquals, 7) - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, "row2_col1", a) + require.Equal(t, "row2_col2\t", b) + require.Equal(t, 7, cc) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &cc) - c.Assert(err, IsNil) - dbt.Check(a, DeepEquals, "row4_col1") - dbt.Check(b, DeepEquals, "\t\t900") - dbt.Check(cc, DeepEquals, 8) - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, "row4_col1", a) + require.Equal(t, "\t\t900", b) + require.Equal(t, 8, cc) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &cc) - c.Assert(err, IsNil) - dbt.Check(a, DeepEquals, "row5_col1") - dbt.Check(b, DeepEquals, "\trow5_col3") - dbt.Check(cc, DeepEquals, 9) - dbt.Check(rows.Next(), IsFalse, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, "row5_col1", a) + require.Equal(t, "\trow5_col3", b) + require.Equal(t, 9, cc) + require.Falsef(t, rows.Next(), "unexpected data") // infile size more than a packet size(16K) - dbt.mustExec("delete from test") + dbt.MustExec("delete from test") _, err = fp.WriteString("\n") - dbt.Assert(err, IsNil) + require.NoError(t, err) for i := 6; i <= 800; i++ { _, err = fp.WriteString(fmt.Sprintf("xxx row%d_col1 - row%d_col2\n", i, i)) - dbt.Assert(err, IsNil) + require.NoError(t, err) } - dbt.mustExec("set @@tidb_dml_batch_size = 3") - rs, err = dbt.db.Exec("load data local infile '/tmp/load_data_test.csv' into table test fields terminated by '\t- ' lines starting by 'xxx ' terminated by '\n'") - dbt.Assert(err, IsNil) + dbt.MustExec("set @@tidb_dml_batch_size = 3") + rs, err = dbt.GetDB().Exec("load data local infile '/tmp/load_data_test.csv' into table test fields terminated by '\t- ' lines starting by 'xxx ' terminated by '\n'") + require.NoError(t, err) lastID, err = rs.LastInsertId() - dbt.Assert(err, IsNil) - dbt.Assert(lastID, Equals, int64(10)) + require.NoError(t, err) + require.Equal(t, int64(10), lastID) affectedRows, err = rs.RowsAffected() - dbt.Assert(err, IsNil) - dbt.Assert(affectedRows, Equals, int64(799)) - rows = dbt.mustQuery("select * from test") - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, int64(799), affectedRows) + rows = dbt.MustQuery("select * from test") + require.Truef(t, rows.Next(), "unexpected data") // don't support lines terminated is "" - dbt.mustExec("set @@tidb_dml_batch_size = 3") - _, err = dbt.db.Exec("load data local infile '/tmp/load_data_test.csv' into table test lines terminated by ''") - dbt.Assert(err, NotNil) + dbt.MustExec("set @@tidb_dml_batch_size = 3") + _, err = dbt.GetDB().Exec("load data local infile '/tmp/load_data_test.csv' into table test lines terminated by ''") + require.NotNil(t, err) // infile doesn't exist - dbt.mustExec("set @@tidb_dml_batch_size = 3") - _, err = dbt.db.Exec("load data local infile '/tmp/nonexistence.csv' into table test") - dbt.Assert(err, NotNil) + dbt.MustExec("set @@tidb_dml_batch_size = 3") + _, err = dbt.GetDB().Exec("load data local infile '/tmp/nonexistence.csv' into table test") + require.NotNil(t, err) }) err = fp.Close() - c.Assert(err, IsNil) + require.NoError(t, err) err = os.Remove(path) - c.Assert(err, IsNil) + require.NoError(t, err) fp, err = os.Create(path) - c.Assert(err, IsNil) - c.Assert(fp, NotNil) + require.NoError(t, err) + require.NotNil(t, fp) // Test mixed unenclosed and enclosed fields. _, err = fp.WriteString( "\"abc\",123\n" + "def,456,\n" + "hig,\"789\",") - c.Assert(err, IsNil) + require.NoError(t, err) - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params["sql_mode"] = "''" - }, "LoadData", func(dbt *DBTest) { - dbt.mustExec("create table test (str varchar(10) default null, i int default null)") - dbt.mustExec("set @@tidb_dml_batch_size = 3") - _, err1 := dbt.db.Exec(`load data local infile '/tmp/load_data_test.csv' into table test FIELDS TERMINATED BY ',' enclosed by '"'`) - dbt.Assert(err1, IsNil) + }, "LoadData", func(dbt *testkit.DBTestKit) { + dbt.MustExec("create table test (str varchar(10) default null, i int default null)") + dbt.MustExec("set @@tidb_dml_batch_size = 3") + _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table test FIELDS TERMINATED BY ',' enclosed by '"'`) + require.NoError(t, err1) var ( str string id int ) - rows := dbt.mustQuery("select * from test") - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + rows := dbt.MustQuery("select * from test") + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&str, &id) - dbt.Check(err, IsNil) - dbt.Check(str, DeepEquals, "abc") - dbt.Check(id, DeepEquals, 123) - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, "abc", str) + require.Equal(t, 123, id) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&str, &id) - c.Assert(err, IsNil) - dbt.Check(str, DeepEquals, "def") - dbt.Check(id, DeepEquals, 456) - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, "def", str) + require.Equal(t, 456, id) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&str, &id) - c.Assert(err, IsNil) - dbt.Check(str, DeepEquals, "hig") - dbt.Check(id, DeepEquals, 789) - dbt.Check(rows.Next(), IsFalse, Commentf("unexpected data")) - dbt.mustExec("delete from test") + require.NoError(t, err) + require.Equal(t, "hig", str) + require.Equal(t, 789, id) + require.Falsef(t, rows.Next(), "unexpected data") + dbt.MustExec("delete from test") }) err = fp.Close() - c.Assert(err, IsNil) + require.NoError(t, err) err = os.Remove(path) - c.Assert(err, IsNil) + require.NoError(t, err) fp, err = os.Create(path) - c.Assert(err, IsNil) - c.Assert(fp, NotNil) + require.NoError(t, err) + require.NotNil(t, fp) // Test irregular csv file. _, err = fp.WriteString( `,\N,NULL,,` + "\n" + "00,0,000000,,\n" + `2003-03-03, 20030303,030303,\N` + "\n") - c.Assert(err, IsNil) + require.NoError(t, err) - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params["sql_mode"] = "''" - }, "LoadData", func(dbt *DBTest) { - dbt.mustExec("create table test (a date, b date, c date not null, d date)") - dbt.mustExec("set @@tidb_dml_batch_size = 3") - _, err1 := dbt.db.Exec(`load data local infile '/tmp/load_data_test.csv' into table test FIELDS TERMINATED BY ','`) - dbt.Assert(err1, IsNil) + }, "LoadData", func(dbt *testkit.DBTestKit) { + dbt.MustExec("create table test (a date, b date, c date not null, d date)") + dbt.MustExec("set @@tidb_dml_batch_size = 3") + _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table test FIELDS TERMINATED BY ','`) + require.NoError(t, err1) var ( a sql.NullString b sql.NullString d sql.NullString c sql.NullString ) - rows := dbt.mustQuery("select * from test") - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + rows := dbt.MustQuery("select * from test") + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &c, &d) - dbt.Check(err, IsNil) - dbt.Check(a.String, Equals, "0000-00-00") - dbt.Check(b.String, Equals, "") - dbt.Check(c.String, Equals, "0000-00-00") - dbt.Check(d.String, Equals, "0000-00-00") - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, "0000-00-00", a.String) + require.Empty(t, b.String) + require.Equal(t, "0000-00-00", c.String) + require.Equal(t, "0000-00-00", d.String) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &c, &d) - dbt.Check(err, IsNil) - dbt.Check(a.String, Equals, "0000-00-00") - dbt.Check(b.String, Equals, "0000-00-00") - dbt.Check(c.String, Equals, "0000-00-00") - dbt.Check(d.String, Equals, "0000-00-00") - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, "0000-00-00", a.String) + require.Equal(t, "0000-00-00", b.String) + require.Equal(t, "0000-00-00", c.String) + require.Equal(t, "0000-00-00", d.String) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &c, &d) - dbt.Check(err, IsNil) - dbt.Check(a.String, Equals, "2003-03-03") - dbt.Check(b.String, Equals, "2003-03-03") - dbt.Check(c.String, Equals, "2003-03-03") - dbt.Check(d.String, Equals, "") - dbt.Check(rows.Next(), IsFalse, Commentf("unexpected data")) - dbt.mustExec("delete from test") + require.NoError(t, err) + require.Equal(t, "2003-03-03", a.String) + require.Equal(t, "2003-03-03", b.String) + require.Equal(t, "2003-03-03", c.String) + require.Equal(t, "", d.String) + require.Falsef(t, rows.Next(), "unexpected data") + dbt.MustExec("delete from test") }) err = fp.Close() - c.Assert(err, IsNil) + require.NoError(t, err) err = os.Remove(path) - c.Assert(err, IsNil) + require.NoError(t, err) fp, err = os.Create(path) - c.Assert(err, IsNil) - c.Assert(fp, NotNil) + require.NoError(t, err) + require.NotNil(t, fp) // Test double enclosed. _, err = fp.WriteString( `"field1","field2"` + "\n" + `"a""b","cd""ef"` + "\n" + `"a"b",c"d"e` + "\n") - c.Assert(err, IsNil) + require.NoError(t, err) - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params["sql_mode"] = "''" - }, "LoadData", func(dbt *DBTest) { - dbt.mustExec("create table test (a varchar(20), b varchar(20))") - dbt.mustExec("set @@tidb_dml_batch_size = 3") - _, err1 := dbt.db.Exec(`load data local infile '/tmp/load_data_test.csv' into table test FIELDS TERMINATED BY ',' enclosed by '"'`) - dbt.Assert(err1, IsNil) + }, "LoadData", func(dbt *testkit.DBTestKit) { + dbt.MustExec("create table test (a varchar(20), b varchar(20))") + dbt.MustExec("set @@tidb_dml_batch_size = 3") + _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table test FIELDS TERMINATED BY ',' enclosed by '"'`) + require.NoError(t, err1) var ( a sql.NullString b sql.NullString ) - rows := dbt.mustQuery("select * from test") - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + rows := dbt.MustQuery("select * from test") + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b) - dbt.Check(err, IsNil) - dbt.Check(a.String, Equals, "field1") - dbt.Check(b.String, Equals, "field2") - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, "field1", a.String) + require.Equal(t, "field2", b.String) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b) - c.Assert(err, IsNil) - dbt.Check(a.String, Equals, `a"b`) - dbt.Check(b.String, Equals, `cd"ef`) - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, `a"b`, a.String) + require.Equal(t, `cd"ef`, b.String) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b) - c.Assert(err, IsNil) - dbt.Check(a.String, Equals, `a"b`) - dbt.Check(b.String, Equals, `c"d"e`) - dbt.Check(rows.Next(), IsFalse, Commentf("unexpected data")) - dbt.mustExec("delete from test") + require.NoError(t, err) + require.Equal(t, `a"b`, a.String) + require.Equal(t, `c"d"e`, b.String) + require.Falsef(t, rows.Next(), "unexpected data") + dbt.MustExec("delete from test") }) err = fp.Close() - c.Assert(err, IsNil) + require.NoError(t, err) err = os.Remove(path) - c.Assert(err, IsNil) + require.NoError(t, err) fp, err = os.Create(path) - c.Assert(err, IsNil) - c.Assert(fp, NotNil) + require.NoError(t, err) + require.NotNil(t, fp) // Test OPTIONALLY _, err = fp.WriteString( `"a,b,c` + "\n" + `"1",2,"3"` + "\n") - c.Assert(err, IsNil) + require.NoError(t, err) - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true - }, "LoadData", func(dbt *DBTest) { - dbt.mustExec("create table test (id INT NOT NULL PRIMARY KEY, b INT, c varchar(10))") - dbt.mustExec("set @@tidb_dml_batch_size = 3") - _, err1 := dbt.db.Exec(`load data local infile '/tmp/load_data_test.csv' into table test FIELDS TERMINATED BY ',' OPTIONALLY ENCLOSED BY '\"' IGNORE 1 LINES`) - dbt.Assert(err1, IsNil) + }, "LoadData", func(dbt *testkit.DBTestKit) { + dbt.MustExec("create table test (id INT NOT NULL PRIMARY KEY, b INT, c varchar(10))") + dbt.MustExec("set @@tidb_dml_batch_size = 3") + _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table test FIELDS TERMINATED BY ',' OPTIONALLY ENCLOSED BY '\"' IGNORE 1 LINES`) + require.NoError(t, err1) var ( a int b int c sql.NullString ) - rows := dbt.mustQuery("select * from test") - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + rows := dbt.MustQuery("select * from test") + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &c) - dbt.Check(err, IsNil) - dbt.Check(a, Equals, 1) - dbt.Check(b, Equals, 2) - dbt.Check(c.String, Equals, "3") - dbt.Check(rows.Next(), IsFalse, Commentf("unexpected data")) - dbt.mustExec("delete from test") + require.NoError(t, err) + require.Equal(t, 1, a) + require.Equal(t, 2, b) + require.Equal(t, "3", c.String) + require.Falsef(t, rows.Next(), "unexpected data") + dbt.MustExec("delete from test") }) // unsupport ClientLocalFiles capability server.capability ^= tmysql.ClientLocalFiles - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true - }, "LoadData", func(dbt *DBTest) { - dbt.mustExec("create table test (a varchar(255), b varchar(255) default 'default value', c int not null auto_increment, primary key(c))") - dbt.mustExec("set @@tidb_dml_batch_size = 3") - _, err = dbt.db.Exec("load data local infile '/tmp/load_data_test.csv' into table test") - dbt.Assert(err, NotNil) - checkErrorCode(c, err, errno.ErrNotAllowedCommand) + }, "LoadData", func(dbt *testkit.DBTestKit) { + dbt.MustExec("create table test (a varchar(255), b varchar(255) default 'default value', c int not null auto_increment, primary key(c))") + dbt.MustExec("set @@tidb_dml_batch_size = 3") + _, err = dbt.GetDB().Exec("load data local infile '/tmp/load_data_test.csv' into table test") + require.Error(t, err) + checkErrorCode(t, err, errno.ErrNotAllowedCommand) }) server.capability |= tmysql.ClientLocalFiles err = fp.Close() - c.Assert(err, IsNil) + require.NoError(t, err) err = os.Remove(path) - c.Assert(err, IsNil) + require.NoError(t, err) fp, err = os.Create(path) - c.Assert(err, IsNil) - c.Assert(fp, NotNil) + require.NoError(t, err) + require.NotNil(t, fp) // Test OPTIONALLY _, err = fp.WriteString( `1,2` + "\n" + `3,4` + "\n") - c.Assert(err, IsNil) + require.NoError(t, err) - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params["sql_mode"] = "''" - }, "LoadData", func(dbt *DBTest) { - dbt.mustExec("drop table if exists pn") - dbt.mustExec("create table pn (c1 int, c2 int)") - dbt.mustExec("set @@tidb_dml_batch_size = 1") - _, err1 := dbt.db.Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ','`) - dbt.Assert(err1, IsNil) + }, "LoadData", func(dbt *testkit.DBTestKit) { + dbt.MustExec("drop table if exists pn") + dbt.MustExec("create table pn (c1 int, c2 int)") + dbt.MustExec("set @@tidb_dml_batch_size = 1") + _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ','`) + require.NoError(t, err1) var ( a int b int ) - rows := dbt.mustQuery("select * from pn") - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + rows := dbt.MustQuery("select * from pn") + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b) - dbt.Check(err, IsNil) - dbt.Check(a, Equals, 1) - dbt.Check(b, Equals, 2) - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, 1, a) + require.Equal(t, 2, b) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b) - dbt.Check(err, IsNil) - dbt.Check(a, Equals, 3) - dbt.Check(b, Equals, 4) - dbt.Check(rows.Next(), IsFalse, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, 3, a) + require.Equal(t, 4, b) + require.Falsef(t, rows.Next(), "unexpected data") // fail error processing test - dbt.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/commitOneTaskErr", "return"), IsNil) - _, err1 = dbt.db.Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ','`) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/commitOneTaskErr", "return")) + _, err1 = dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ','`) mysqlErr, ok := err1.(*mysql.MySQLError) - dbt.Assert(ok, IsTrue) - dbt.Assert(mysqlErr.Message, Equals, "mock commit one task error") - dbt.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/commitOneTaskErr"), IsNil) + require.True(t, ok) + require.Equal(t, "mock commit one task error", mysqlErr.Message) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/commitOneTaskErr")) - dbt.mustExec("drop table if exists pn") + dbt.MustExec("drop table if exists pn") }) err = fp.Close() - c.Assert(err, IsNil) + require.NoError(t, err) err = os.Remove(path) - c.Assert(err, IsNil) + require.NoError(t, err) fp, err = os.Create(path) - c.Assert(err, IsNil) - c.Assert(fp, NotNil) + require.NoError(t, err) + require.NotNil(t, fp) // Test Column List Specification _, err = fp.WriteString( `1,2` + "\n" + `3,4` + "\n") - c.Assert(err, IsNil) + require.NoError(t, err) - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params["sql_mode"] = "''" - }, "LoadData", func(dbt *DBTest) { - dbt.mustExec("drop table if exists pn") - dbt.mustExec("create table pn (c1 int, c2 int)") - dbt.mustExec("set @@tidb_dml_batch_size = 1") - _, err1 := dbt.db.Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ',' (c1, c2)`) - dbt.Assert(err1, IsNil) + }, "LoadData", func(dbt *testkit.DBTestKit) { + dbt.MustExec("drop table if exists pn") + dbt.MustExec("create table pn (c1 int, c2 int)") + dbt.MustExec("set @@tidb_dml_batch_size = 1") + _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ',' (c1, c2)`) + require.NoError(t, err1) var ( a int b int ) - rows := dbt.mustQuery("select * from pn") - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + rows := dbt.MustQuery("select * from pn") + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b) - dbt.Check(err, IsNil) - dbt.Check(a, Equals, 1) - dbt.Check(b, Equals, 2) - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, 1, a) + require.Equal(t, 2, b) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b) - dbt.Check(err, IsNil) - dbt.Check(a, Equals, 3) - dbt.Check(b, Equals, 4) - dbt.Check(rows.Next(), IsFalse, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, 3, a) + require.Equal(t, 4, b) + require.Falsef(t, rows.Next(), "unexpected data") - dbt.mustExec("drop table if exists pn") + dbt.MustExec("drop table if exists pn") }) err = fp.Close() - c.Assert(err, IsNil) + require.NoError(t, err) err = os.Remove(path) - c.Assert(err, IsNil) + require.NoError(t, err) fp, err = os.Create(path) - c.Assert(err, IsNil) - c.Assert(fp, NotNil) + require.NoError(t, err) + require.NotNil(t, fp) // Test Column List Specification _, err = fp.WriteString( `1,2,3` + "\n" + `4,5,6` + "\n") - c.Assert(err, IsNil) + require.NoError(t, err) - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params["sql_mode"] = "''" - }, "LoadData", func(dbt *DBTest) { - dbt.mustExec("drop table if exists pn") - dbt.mustExec("create table pn (c1 int, c2 int, c3 int)") - dbt.mustExec("set @@tidb_dml_batch_size = 1") - _, err1 := dbt.db.Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ',' (c1, @dummy)`) - dbt.Assert(err1, IsNil) + }, "LoadData", func(dbt *testkit.DBTestKit) { + dbt.MustExec("drop table if exists pn") + dbt.MustExec("create table pn (c1 int, c2 int, c3 int)") + dbt.MustExec("set @@tidb_dml_batch_size = 1") + _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ',' (c1, @dummy)`) + require.NoError(t, err1) var ( a int b sql.NullString c sql.NullString ) - rows := dbt.mustQuery("select * from pn") - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + rows := dbt.MustQuery("select * from pn") + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &c) - dbt.Check(err, IsNil) - dbt.Check(a, Equals, 1) - dbt.Check(b.String, Equals, "") - dbt.Check(c.String, Equals, "") - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, 1, a) + require.Empty(t, b.String) + require.Empty(t, c.String) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &c) - dbt.Check(err, IsNil) - dbt.Check(a, Equals, 4) - dbt.Check(b.String, Equals, "") - dbt.Check(c.String, Equals, "") - dbt.Check(rows.Next(), IsFalse, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, 4, a) + require.Empty(t, b.String) + require.Empty(t, c.String) + require.Falsef(t, rows.Next(), "unexpected data") - dbt.mustExec("drop table if exists pn") + dbt.MustExec("drop table if exists pn") }) err = fp.Close() - c.Assert(err, IsNil) + require.NoError(t, err) err = os.Remove(path) - c.Assert(err, IsNil) + require.NoError(t, err) fp, err = os.Create(path) - c.Assert(err, IsNil) - c.Assert(fp, NotNil) + require.NoError(t, err) + require.NotNil(t, fp) // Test Input Preprocessing _, err = fp.WriteString( `1,2,3` + "\n" + `4,5,6` + "\n") - c.Assert(err, IsNil) + require.NoError(t, err) - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true config.Params["sql_mode"] = "''" - }, "LoadData", func(dbt *DBTest) { - dbt.mustExec("drop table if exists pn") - dbt.mustExec("create table pn (c1 int, c2 int, c3 int)") - dbt.mustExec("set @@tidb_dml_batch_size = 1") - _, err1 := dbt.db.Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ',' (c1, @val1, @val2) SET c3 = @val2 * 100, c2 = CAST(@val1 AS UNSIGNED)`) - dbt.Assert(err1, IsNil) + }, "LoadData", func(dbt *testkit.DBTestKit) { + dbt.MustExec("drop table if exists pn") + dbt.MustExec("create table pn (c1 int, c2 int, c3 int)") + dbt.MustExec("set @@tidb_dml_batch_size = 1") + _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ',' (c1, @val1, @val2) SET c3 = @val2 * 100, c2 = CAST(@val1 AS UNSIGNED)`) + require.NoError(t, err1) var ( a int b int c int ) - rows := dbt.mustQuery("select * from pn") - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + rows := dbt.MustQuery("select * from pn") + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &c) - dbt.Check(err, IsNil) - dbt.Check(a, Equals, 1) - dbt.Check(b, Equals, 2) - dbt.Check(c, Equals, 300) - dbt.Check(rows.Next(), IsTrue, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, 1, a) + require.Equal(t, 2, b) + require.Equal(t, 300, c) + require.Truef(t, rows.Next(), "unexpected data") err = rows.Scan(&a, &b, &c) - dbt.Check(err, IsNil) - dbt.Check(a, Equals, 4) - dbt.Check(b, Equals, 5) - dbt.Check(c, Equals, 600) - dbt.Check(rows.Next(), IsFalse, Commentf("unexpected data")) + require.NoError(t, err) + require.Equal(t, 4, a) + require.Equal(t, 5, b) + require.Equal(t, 600, c) + require.Falsef(t, rows.Next(), "unexpected data") - dbt.mustExec("drop table if exists pn") + dbt.MustExec("drop table if exists pn") }) } -func (cli *testServerClient) runTestConcurrentUpdate(c *C) { +func (cli *testServerClient) runTestConcurrentUpdate(t *testing.T) { dbName := "Concurrent" - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.Params["sql_mode"] = "''" - }, dbName, func(dbt *DBTest) { - dbt.mustExec("drop table if exists test2") - dbt.mustExec("create table test2 (a int, b int)") - dbt.mustExec("insert test2 values (1, 1)") - dbt.mustExec("set @@tidb_disable_txn_auto_retry = 0") - - txn1, err := dbt.db.Begin() - c.Assert(err, IsNil) + }, dbName, func(dbt *testkit.DBTestKit) { + dbt.MustExec("drop table if exists test2") + dbt.MustExec("create table test2 (a int, b int)") + dbt.MustExec("insert test2 values (1, 1)") + dbt.MustExec("set @@tidb_disable_txn_auto_retry = 0") + + txn1, err := dbt.GetDB().Begin() + require.NoError(t, err) _, err = txn1.Exec(fmt.Sprintf("USE `%s`;", dbName)) - c.Assert(err, IsNil) + require.NoError(t, err) - txn2, err := dbt.db.Begin() - c.Assert(err, IsNil) + txn2, err := dbt.GetDB().Begin() + require.NoError(t, err) _, err = txn2.Exec(fmt.Sprintf("USE `%s`;", dbName)) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = txn2.Exec("update test2 set a = a + 1 where b = 1") - c.Assert(err, IsNil) + require.NoError(t, err) err = txn2.Commit() - c.Assert(err, IsNil) + require.NoError(t, err) _, err = txn1.Exec("update test2 set a = a + 1 where b = 1") - c.Assert(err, IsNil) + require.NoError(t, err) err = txn1.Commit() - c.Assert(err, IsNil) + require.NoError(t, err) }) } -func (cli *testServerClient) runTestExplainForConn(c *C) { - cli.runTestsOnNewDB(c, nil, "explain_for_conn", func(dbt *DBTest) { - dbt.mustExec("drop table if exists t") - dbt.mustExec("create table t (a int key, b int)") - dbt.mustExec("insert t values (1, 1)") - rows := dbt.mustQuery("select connection_id();") - c.Assert(rows.Next(), IsTrue) +func (cli *testServerClient) runTestExplainForConn(t *testing.T) { + cli.runTestsOnNewDB(t, nil, "explain_for_conn", func(dbt *testkit.DBTestKit) { + dbt.MustExec("drop table if exists t") + dbt.MustExec("create table t (a int key, b int)") + dbt.MustExec("insert t values (1, 1)") + rows := dbt.MustQuery("select connection_id();") + require.True(t, rows.Next()) var connID int64 err := rows.Scan(&connID) - c.Assert(err, IsNil) - c.Assert(rows.Close(), IsNil) - dbt.mustQuery("select * from t where a=1") - rows = dbt.mustQuery("explain for connection " + strconv.Itoa(int(connID))) - c.Assert(rows.Next(), IsTrue) + require.NoError(t, err) + require.NoError(t, rows.Close()) + dbt.MustQuery("select * from t where a=1") + rows = dbt.MustQuery("explain for connection " + strconv.Itoa(int(connID))) + require.True(t, rows.Next()) row := make([]string, 9) err = rows.Scan(&row[0], &row[1], &row[2], &row[3], &row[4], &row[5], &row[6], &row[7], &row[8]) - c.Assert(err, IsNil) - c.Assert(strings.Join(row, ","), Matches, "Point_Get_1,1.00,1,root,table:t,time.*loop.*handle:1.*") - c.Assert(rows.Close(), IsNil) + require.NoError(t, err) + require.Regexp(t, "Point_Get_1,1.00,1,root,table:t,time.*loop.*handle:1.*", strings.Join(row, ",")) + require.NoError(t, rows.Close()) }) } -func (cli *testServerClient) runTestErrorCode(c *C) { - cli.runTestsOnNewDB(c, nil, "ErrorCode", func(dbt *DBTest) { - dbt.mustExec("create table test (c int PRIMARY KEY);") - dbt.mustExec("insert into test values (1);") - txn1, err := dbt.db.Begin() - c.Assert(err, IsNil) +func (cli *testServerClient) runTestErrorCode(t *testing.T) { + cli.runTestsOnNewDB(t, nil, "ErrorCode", func(dbt *testkit.DBTestKit) { + dbt.MustExec("create table test (c int PRIMARY KEY);") + dbt.MustExec("insert into test values (1);") + txn1, err := dbt.GetDB().Begin() + require.NoError(t, err) _, err = txn1.Exec("insert into test values(1)") - c.Assert(err, IsNil) + require.NoError(t, err) err = txn1.Commit() - checkErrorCode(c, err, errno.ErrDupEntry) + checkErrorCode(t, err, errno.ErrDupEntry) // Schema errors - txn2, err := dbt.db.Begin() - c.Assert(err, IsNil) + txn2, err := dbt.GetDB().Begin() + require.NoError(t, err) _, err = txn2.Exec("use db_not_exists;") - checkErrorCode(c, err, errno.ErrBadDB) + checkErrorCode(t, err, errno.ErrBadDB) _, err = txn2.Exec("select * from tbl_not_exists;") - checkErrorCode(c, err, errno.ErrNoSuchTable) + checkErrorCode(t, err, errno.ErrNoSuchTable) _, err = txn2.Exec("create database test;") // Make tests stable. Some times the error may be the ErrInfoSchemaChanged. - checkErrorCode(c, err, errno.ErrDBCreateExists, errno.ErrInfoSchemaChanged) + checkErrorCode(t, err, errno.ErrDBCreateExists, errno.ErrInfoSchemaChanged) _, err = txn2.Exec("create database aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa;") - checkErrorCode(c, err, errno.ErrTooLongIdent, errno.ErrInfoSchemaChanged) + checkErrorCode(t, err, errno.ErrTooLongIdent, errno.ErrInfoSchemaChanged) _, err = txn2.Exec("create table test (c int);") - checkErrorCode(c, err, errno.ErrTableExists, errno.ErrInfoSchemaChanged) + checkErrorCode(t, err, errno.ErrTableExists, errno.ErrInfoSchemaChanged) _, err = txn2.Exec("drop table unknown_table;") - checkErrorCode(c, err, errno.ErrBadTable, errno.ErrInfoSchemaChanged) + checkErrorCode(t, err, errno.ErrBadTable, errno.ErrInfoSchemaChanged) _, err = txn2.Exec("drop database unknown_db;") - checkErrorCode(c, err, errno.ErrDBDropExists, errno.ErrInfoSchemaChanged) + checkErrorCode(t, err, errno.ErrDBDropExists, errno.ErrInfoSchemaChanged) _, err = txn2.Exec("create table aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa (a int);") - checkErrorCode(c, err, errno.ErrTooLongIdent, errno.ErrInfoSchemaChanged) + checkErrorCode(t, err, errno.ErrTooLongIdent, errno.ErrInfoSchemaChanged) _, err = txn2.Exec("create table long_column_table (aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa int);") - checkErrorCode(c, err, errno.ErrTooLongIdent, errno.ErrInfoSchemaChanged) + checkErrorCode(t, err, errno.ErrTooLongIdent, errno.ErrInfoSchemaChanged) _, err = txn2.Exec("alter table test add aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa int;") - checkErrorCode(c, err, errno.ErrTooLongIdent, errno.ErrInfoSchemaChanged) + checkErrorCode(t, err, errno.ErrTooLongIdent, errno.ErrInfoSchemaChanged) // Optimizer errors _, err = txn2.Exec("select *, * from test;") - checkErrorCode(c, err, errno.ErrInvalidWildCard) + checkErrorCode(t, err, errno.ErrInvalidWildCard) _, err = txn2.Exec("select row(1, 2) > 1;") - checkErrorCode(c, err, errno.ErrOperandColumns) + checkErrorCode(t, err, errno.ErrOperandColumns) _, err = txn2.Exec("select * from test order by row(c, c);") - checkErrorCode(c, err, errno.ErrOperandColumns) + checkErrorCode(t, err, errno.ErrOperandColumns) // Variable errors _, err = txn2.Exec("select @@unknown_sys_var;") - checkErrorCode(c, err, errno.ErrUnknownSystemVariable) + checkErrorCode(t, err, errno.ErrUnknownSystemVariable) _, err = txn2.Exec("set @@unknown_sys_var='1';") - checkErrorCode(c, err, errno.ErrUnknownSystemVariable) + checkErrorCode(t, err, errno.ErrUnknownSystemVariable) // Expression errors _, err = txn2.Exec("select greatest(2);") - checkErrorCode(c, err, errno.ErrWrongParamcountToNativeFct) + checkErrorCode(t, err, errno.ErrWrongParamcountToNativeFct) }) } -func checkErrorCode(c *C, e error, codes ...uint16) { +func checkErrorCode(t *testing.T, e error, codes ...uint16) { me, ok := e.(*mysql.MySQLError) - c.Assert(ok, IsTrue, Commentf("err: %v", e)) + require.Truef(t, ok, "err: %v", e) if len(codes) == 1 { - c.Assert(me.Number, Equals, codes[0]) + require.Equal(t, codes[0], me.Number) } isMatchCode := false for _, code := range codes { @@ -1530,104 +1488,104 @@ func checkErrorCode(c *C, e error, codes ...uint16) { break } } - c.Assert(isMatchCode, IsTrue, Commentf("got err %v, expected err codes %v", me, codes)) + require.Truef(t, isMatchCode, "got err %v, expected err codes %v", me, codes) } -func (cli *testServerClient) runTestAuth(c *C) { - cli.runTests(c, nil, func(dbt *DBTest) { - dbt.mustExec(`CREATE USER 'authtest'@'%' IDENTIFIED BY '123';`) - dbt.mustExec(`CREATE ROLE 'authtest_r1'@'%';`) - dbt.mustExec(`GRANT ALL on test.* to 'authtest'`) - dbt.mustExec(`GRANT authtest_r1 to 'authtest'`) - dbt.mustExec(`SET DEFAULT ROLE authtest_r1 TO authtest`) +func (cli *testServerClient) runTestAuth(t *testing.T) { + cli.runTests(t, nil, func(dbt *testkit.DBTestKit) { + dbt.MustExec(`CREATE USER 'authtest'@'%' IDENTIFIED BY '123';`) + dbt.MustExec(`CREATE ROLE 'authtest_r1'@'%';`) + dbt.MustExec(`GRANT ALL on test.* to 'authtest'`) + dbt.MustExec(`GRANT authtest_r1 to 'authtest'`) + dbt.MustExec(`SET DEFAULT ROLE authtest_r1 TO authtest`) }) - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.User = "authtest" config.Passwd = "123" - }, func(dbt *DBTest) { - dbt.mustExec(`USE information_schema;`) + }, func(dbt *testkit.DBTestKit) { + dbt.MustExec(`USE information_schema;`) }) db, err := sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { config.User = "authtest" config.Passwd = "456" })) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = db.Query("USE information_schema;") - c.Assert(err, NotNil, Commentf("Wrong password should be failed")) + require.NotNilf(t, err, "Wrong password should be failed") err = db.Close() - c.Assert(err, IsNil) + require.NoError(t, err) // Test for loading active roles. db, err = sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { config.User = "authtest" config.Passwd = "123" })) - c.Assert(err, IsNil) + require.NoError(t, err) rows, err := db.Query("select current_role;") - c.Assert(err, IsNil) - c.Assert(rows.Next(), IsTrue) + require.NoError(t, err) + require.True(t, rows.Next()) var outA string err = rows.Scan(&outA) - c.Assert(err, IsNil) - c.Assert(outA, Equals, "`authtest_r1`@`%`") + require.NoError(t, err) + require.Equal(t, "`authtest_r1`@`%`", outA) err = db.Close() - c.Assert(err, IsNil) + require.NoError(t, err) // Test login use IP that not exists in mysql.user. - cli.runTests(c, nil, func(dbt *DBTest) { - dbt.mustExec(`CREATE USER 'authtest2'@'localhost' IDENTIFIED BY '123';`) - dbt.mustExec(`GRANT ALL on test.* to 'authtest2'@'localhost'`) + cli.runTests(t, nil, func(dbt *testkit.DBTestKit) { + dbt.MustExec(`CREATE USER 'authtest2'@'localhost' IDENTIFIED BY '123';`) + dbt.MustExec(`GRANT ALL on test.* to 'authtest2'@'localhost'`) }) - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.User = "authtest2" config.Passwd = "123" - }, func(dbt *DBTest) { - dbt.mustExec(`USE information_schema;`) + }, func(dbt *testkit.DBTestKit) { + dbt.MustExec(`USE information_schema;`) }) } -func (cli *testServerClient) runTestIssue3662(c *C) { +func (cli *testServerClient) runTestIssue3662(t *testing.T) { db, err := sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { config.DBName = "non_existing_schema" })) - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := db.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() // According to documentation, "Open may just validate its arguments without // creating a connection to the database. To verify that the data source name // is valid, call Ping." err = db.Ping() - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "Error 1049: Unknown database 'non_existing_schema'") + require.Error(t, err) + require.Equal(t, "Error 1049: Unknown database 'non_existing_schema'", err.Error()) } -func (cli *testServerClient) runTestIssue3680(c *C) { +func (cli *testServerClient) runTestIssue3680(t *testing.T) { db, err := sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { config.User = "non_existing_user" })) - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := db.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() // According to documentation, "Open may just validate its arguments without // creating a connection to the database. To verify that the data source name // is valid, call Ping." err = db.Ping() - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "Error 1045: Access denied for user 'non_existing_user'@'127.0.0.1' (using password: NO)") + require.Error(t, err) + require.Equal(t, "Error 1045: Access denied for user 'non_existing_user'@'127.0.0.1' (using password: NO)", err.Error()) } -func (cli *testServerClient) runTestIssue22646(c *C) { - cli.runTests(c, nil, func(dbt *DBTest) { +func (cli *testServerClient) runTestIssue22646(t *testing.T) { + cli.runTests(t, nil, func(dbt *testkit.DBTestKit) { c1 := make(chan string, 1) go func() { - dbt.mustExec(``) // empty query. + dbt.MustExec(``) // empty query. c1 <- "success" }() select { @@ -1639,232 +1597,231 @@ func (cli *testServerClient) runTestIssue22646(c *C) { }) } -func (cli *testServerClient) runTestIssue3682(c *C) { - cli.runTests(c, nil, func(dbt *DBTest) { - dbt.mustExec(`CREATE USER 'issue3682'@'%' IDENTIFIED BY '123';`) - dbt.mustExec(`GRANT ALL on test.* to 'issue3682'`) - dbt.mustExec(`GRANT ALL on mysql.* to 'issue3682'`) +func (cli *testServerClient) runTestIssue3682(t *testing.T) { + cli.runTests(t, nil, func(dbt *testkit.DBTestKit) { + dbt.MustExec(`CREATE USER 'issue3682'@'%' IDENTIFIED BY '123';`) + dbt.MustExec(`GRANT ALL on test.* to 'issue3682'`) + dbt.MustExec(`GRANT ALL on mysql.* to 'issue3682'`) }) - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.User = "issue3682" config.Passwd = "123" - }, func(dbt *DBTest) { - dbt.mustExec(`USE mysql;`) + }, func(dbt *testkit.DBTestKit) { + dbt.MustExec(`USE mysql;`) }) db, err := sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { config.User = "issue3682" config.Passwd = "wrong_password" config.DBName = "non_existing_schema" })) - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := db.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() err = db.Ping() - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "Error 1045: Access denied for user 'issue3682'@'127.0.0.1' (using password: YES)") + require.Error(t, err) + require.Equal(t, "Error 1045: Access denied for user 'issue3682'@'127.0.0.1' (using password: YES)", err.Error()) } -func (cli *testServerClient) runTestDBNameEscape(c *C) { - cli.runTests(c, nil, func(dbt *DBTest) { - dbt.mustExec("CREATE DATABASE `aa-a`;") +func (cli *testServerClient) runTestDBNameEscape(t *testing.T) { + cli.runTests(t, nil, func(dbt *testkit.DBTestKit) { + dbt.MustExec("CREATE DATABASE `aa-a`;") }) - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.DBName = "aa-a" - }, func(dbt *DBTest) { - dbt.mustExec(`USE mysql;`) - dbt.mustExec("DROP DATABASE `aa-a`") + }, func(dbt *testkit.DBTestKit) { + dbt.MustExec(`USE mysql;`) + dbt.MustExec("DROP DATABASE `aa-a`") }) } -func (cli *testServerClient) runTestResultFieldTableIsNull(c *C) { - cli.runTestsOnNewDB(c, func(config *mysql.Config) { +func (cli *testServerClient) runTestResultFieldTableIsNull(t *testing.T) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.Params["sql_mode"] = "''" - }, "ResultFieldTableIsNull", func(dbt *DBTest) { - dbt.mustExec("drop table if exists test;") - dbt.mustExec("create table test (c int);") - dbt.mustExec("explain select * from test;") + }, "ResultFieldTableIsNull", func(dbt *testkit.DBTestKit) { + dbt.MustExec("drop table if exists test;") + dbt.MustExec("create table test (c int);") + dbt.MustExec("explain select * from test;") }) } -func (cli *testServerClient) runTestStatusAPI(c *C) { +func (cli *testServerClient) runTestStatusAPI(t *testing.T) { resp, err := cli.fetchStatus("/status") - c.Assert(err, IsNil) + require.NoError(t, err) defer resp.Body.Close() decoder := json.NewDecoder(resp.Body) var data status err = decoder.Decode(&data) - c.Assert(err, IsNil) - c.Assert(data.Version, Equals, tmysql.ServerVersion) - c.Assert(data.GitHash, Equals, versioninfo.TiDBGitHash) + require.NoError(t, err) + require.Equal(t, tmysql.ServerVersion, data.Version) + require.Equal(t, versioninfo.TiDBGitHash, data.GitHash) } // The golang sql driver (and most drivers) should have multi-statement // disabled by default for security reasons. Lets ensure that the behavior // is correct. -func (cli *testServerClient) runFailedTestMultiStatements(c *C) { - cli.runTestsOnNewDB(c, nil, "FailedMultiStatements", func(dbt *DBTest) { +func (cli *testServerClient) runFailedTestMultiStatements(t *testing.T) { + cli.runTestsOnNewDB(t, nil, "FailedMultiStatements", func(dbt *testkit.DBTestKit) { // Default is now OFF in new installations. // It is still WARN in upgrade installations (for now) - _, err := dbt.db.Exec("SELECT 1; SELECT 1; SELECT 2; SELECT 3;") - c.Assert(err.Error(), Equals, "Error 8130: client has multi-statement capability disabled. Run SET GLOBAL tidb_multi_statement_mode='ON' after you understand the security risk") + _, err := dbt.GetDB().Exec("SELECT 1; SELECT 1; SELECT 2; SELECT 3;") + require.Equal(t, "Error 8130: client has multi-statement capability disabled. Run SET GLOBAL tidb_multi_statement_mode='ON' after you understand the security risk", err.Error()) // Change to WARN (legacy mode) - dbt.mustExec("SET tidb_multi_statement_mode='WARN'") - dbt.mustExec("CREATE TABLE `test` (`id` int(11) NOT NULL, `value` int(11) NOT NULL) ") - res := dbt.mustExec("INSERT INTO test VALUES (1, 1)") + dbt.MustExec("SET tidb_multi_statement_mode='WARN'") + dbt.MustExec("CREATE TABLE `test` (`id` int(11) NOT NULL, `value` int(11) NOT NULL) ") + res := dbt.MustExec("INSERT INTO test VALUES (1, 1)") count, err := res.RowsAffected() - c.Assert(err, IsNil, Commentf("res.RowsAffected() returned error")) - c.Assert(count, Equals, int64(1)) - res = dbt.mustExec("UPDATE test SET value = 3 WHERE id = 1; UPDATE test SET value = 4 WHERE id = 1; UPDATE test SET value = 5 WHERE id = 1;") + require.NoErrorf(t, err, "res.RowsAffected() returned error") + require.Equal(t, int64(1), count) + res = dbt.MustExec("UPDATE test SET value = 3 WHERE id = 1; UPDATE test SET value = 4 WHERE id = 1; UPDATE test SET value = 5 WHERE id = 1;") count, err = res.RowsAffected() - c.Assert(err, IsNil, Commentf("res.RowsAffected() returned error")) - c.Assert(count, Equals, int64(1)) - rows := dbt.mustQuery("show warnings") - cli.checkRows(c, rows, "Warning 8130 client has multi-statement capability disabled. Run SET GLOBAL tidb_multi_statement_mode='ON' after you understand the security risk") + require.NoErrorf(t, err, "res.RowsAffected() returned error") + require.Equal(t, int64(1), count) + rows := dbt.MustQuery("show warnings") + cli.checkRows(t, rows, "Warning 8130 client has multi-statement capability disabled. Run SET GLOBAL tidb_multi_statement_mode='ON' after you understand the security risk") var out int - rows = dbt.mustQuery("SELECT value FROM test WHERE id=1;") + rows = dbt.MustQuery("SELECT value FROM test WHERE id=1;") if rows.Next() { err = rows.Scan(&out) - c.Assert(err, IsNil) - c.Assert(out, Equals, 5) + require.NoError(t, err) + require.Equal(t, 5, out) if rows.Next() { - dbt.Error("unexpected data") + require.Fail(t, "unexpected data") } } else { - dbt.Error("no data") + require.Fail(t, "no data") } // Change to ON = Fully supported, TiDB legacy. No warnings or Errors. - dbt.mustExec("SET tidb_multi_statement_mode='ON';") - dbt.mustExec("DROP TABLE IF EXISTS test") - dbt.mustExec("CREATE TABLE `test` (`id` int(11) NOT NULL, `value` int(11) NOT NULL) ") - res = dbt.mustExec("INSERT INTO test VALUES (1, 1)") + dbt.MustExec("SET tidb_multi_statement_mode='ON';") + dbt.MustExec("DROP TABLE IF EXISTS test") + dbt.MustExec("CREATE TABLE `test` (`id` int(11) NOT NULL, `value` int(11) NOT NULL) ") + res = dbt.MustExec("INSERT INTO test VALUES (1, 1)") count, err = res.RowsAffected() - c.Assert(err, IsNil, Commentf("res.RowsAffected() returned error")) - c.Assert(count, Equals, int64(1)) - res = dbt.mustExec("update test SET value = 3 WHERE id = 1; UPDATE test SET value = 4 WHERE id = 1; UPDATE test SET value = 5 WHERE id = 1;") + require.NoErrorf(t, err, "res.RowsAffected() returned error") + require.Equal(t, int64(1), count) + res = dbt.MustExec("update test SET value = 3 WHERE id = 1; UPDATE test SET value = 4 WHERE id = 1; UPDATE test SET value = 5 WHERE id = 1;") count, err = res.RowsAffected() - c.Assert(err, IsNil, Commentf("res.RowsAffected() returned error")) - c.Assert(count, Equals, int64(1)) - rows = dbt.mustQuery("SELECT value FROM test WHERE id=1;") + require.NoErrorf(t, err, "res.RowsAffected() returned error") + require.Equal(t, int64(1), count) + rows = dbt.MustQuery("SELECT value FROM test WHERE id=1;") if rows.Next() { err = rows.Scan(&out) - c.Assert(err, IsNil) - c.Assert(out, Equals, 5) + require.NoError(t, err) + require.Equal(t, 5, out) if rows.Next() { - dbt.Error("unexpected data") + require.Fail(t, "unexpected data") } } else { - dbt.Error("no data") + require.Fail(t, "no data") } - }) } -func (cli *testServerClient) runTestMultiStatements(c *C) { +func (cli *testServerClient) runTestMultiStatements(t *testing.T) { - cli.runTestsOnNewDB(c, func(config *mysql.Config) { + cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.Params["multiStatements"] = "true" - }, "MultiStatements", func(dbt *DBTest) { + }, "MultiStatements", func(dbt *testkit.DBTestKit) { // Create Table - dbt.mustExec("CREATE TABLE `test` (`id` int(11) NOT NULL, `value` int(11) NOT NULL) ") + dbt.MustExec("CREATE TABLE `test` (`id` int(11) NOT NULL, `value` int(11) NOT NULL) ") // Create Data - res := dbt.mustExec("INSERT INTO test VALUES (1, 1)") + res := dbt.MustExec("INSERT INTO test VALUES (1, 1)") count, err := res.RowsAffected() - c.Assert(err, IsNil, Commentf("res.RowsAffected() returned error")) - c.Assert(count, Equals, int64(1)) + require.NoErrorf(t, err, "res.RowsAffected() returned error") + require.Equal(t, int64(1), count) // Update - res = dbt.mustExec("UPDATE test SET value = 3 WHERE id = 1; UPDATE test SET value = 4 WHERE id = 1; UPDATE test SET value = 5 WHERE id = 1;") + res = dbt.MustExec("UPDATE test SET value = 3 WHERE id = 1; UPDATE test SET value = 4 WHERE id = 1; UPDATE test SET value = 5 WHERE id = 1;") count, err = res.RowsAffected() - c.Assert(err, IsNil, Commentf("res.RowsAffected() returned error")) - c.Assert(count, Equals, int64(1)) + require.NoErrorf(t, err, "res.RowsAffected() returned error") + require.Equal(t, int64(1), count) // Read var out int - rows := dbt.mustQuery("SELECT value FROM test WHERE id=1;") + rows := dbt.MustQuery("SELECT value FROM test WHERE id=1;") if rows.Next() { err = rows.Scan(&out) - c.Assert(err, IsNil) - c.Assert(out, Equals, 5) + require.NoError(t, err) + require.Equal(t, 5, out) if rows.Next() { - dbt.Error("unexpected data") + require.Fail(t, "unexpected data") } } else { - dbt.Error("no data") + require.Fail(t, "no data") } // Test issue #26688 // First we "reset" the CurrentDB by using a database and then dropping it. - dbt.mustExec("CREATE DATABASE dropme") - dbt.mustExec("USE dropme") - dbt.mustExec("DROP DATABASE dropme") + dbt.MustExec("CREATE DATABASE dropme") + dbt.MustExec("USE dropme") + dbt.MustExec("DROP DATABASE dropme") var usedb string - rows = dbt.mustQuery("SELECT IFNULL(DATABASE(),'success')") + rows = dbt.MustQuery("SELECT IFNULL(DATABASE(),'success')") if rows.Next() { err = rows.Scan(&usedb) - c.Assert(err, IsNil) - c.Assert(usedb, Equals, "success") + require.NoError(t, err) + require.Equal(t, "success", usedb) } else { - dbt.Error("no database() result") + require.Fail(t, "no database() result") } // Because no DB is selected, if the use multistmtuse is not successful, then // the create table + drop table statements will return errors. - dbt.mustExec("CREATE DATABASE multistmtuse") - dbt.mustExec("use multistmtuse; create table if not exists t1 (id int); drop table t1;") + dbt.MustExec("CREATE DATABASE multistmtuse") + dbt.MustExec("use multistmtuse; create table if not exists t1 (id int); drop table t1;") }) } -func (cli *testServerClient) runTestStmtCount(t *C) { - cli.runTestsOnNewDB(t, nil, "StatementCount", func(dbt *DBTest) { +func (cli *testServerClient) runTestStmtCount(t *testing.T) { + cli.runTestsOnNewDB(t, nil, "StatementCount", func(dbt *testkit.DBTestKit) { originStmtCnt := getStmtCnt(string(cli.getMetrics(t))) - dbt.mustExec("create table test (a int)") + dbt.MustExec("create table test (a int)") - dbt.mustExec("insert into test values(1)") - dbt.mustExec("insert into test values(2)") - dbt.mustExec("insert into test values(3)") - dbt.mustExec("insert into test values(4)") - dbt.mustExec("insert into test values(5)") + dbt.MustExec("insert into test values(1)") + dbt.MustExec("insert into test values(2)") + dbt.MustExec("insert into test values(3)") + dbt.MustExec("insert into test values(4)") + dbt.MustExec("insert into test values(5)") - dbt.mustExec("delete from test where a = 3") - dbt.mustExec("update test set a = 2 where a = 1") - dbt.mustExec("select * from test") - dbt.mustExec("select 2") + dbt.MustExec("delete from test where a = 3") + dbt.MustExec("update test set a = 2 where a = 1") + dbt.MustExec("select * from test") + dbt.MustExec("select 2") - dbt.mustExec("prepare stmt1 from 'update test set a = 1 where a = 2'") - dbt.mustExec("execute stmt1") - dbt.mustExec("prepare stmt2 from 'select * from test'") - dbt.mustExec("execute stmt2") - dbt.mustExec("replace into test(a) values(6);") + dbt.MustExec("prepare stmt1 from 'update test set a = 1 where a = 2'") + dbt.MustExec("execute stmt1") + dbt.MustExec("prepare stmt2 from 'select * from test'") + dbt.MustExec("execute stmt2") + dbt.MustExec("replace into test(a) values(6);") currentStmtCnt := getStmtCnt(string(cli.getMetrics(t))) - t.Assert(currentStmtCnt["CreateTable"], Equals, originStmtCnt["CreateTable"]+1) - t.Assert(currentStmtCnt["Insert"], Equals, originStmtCnt["Insert"]+5) - t.Assert(currentStmtCnt["Delete"], Equals, originStmtCnt["Delete"]+1) - t.Assert(currentStmtCnt["Update"], Equals, originStmtCnt["Update"]+1) - t.Assert(currentStmtCnt["Select"], Equals, originStmtCnt["Select"]+2) - t.Assert(currentStmtCnt["Prepare"], Equals, originStmtCnt["Prepare"]+2) - t.Assert(currentStmtCnt["Execute"], Equals, originStmtCnt["Execute"]+2) - t.Assert(currentStmtCnt["Replace"], Equals, originStmtCnt["Replace"]+1) + require.Equal(t, originStmtCnt["CreateTable"]+1, currentStmtCnt["CreateTable"]) + require.Equal(t, originStmtCnt["Insert"]+5, currentStmtCnt["Insert"]) + require.Equal(t, originStmtCnt["Delete"]+1, currentStmtCnt["Delete"]) + require.Equal(t, originStmtCnt["Update"]+1, currentStmtCnt["Update"]) + require.Equal(t, originStmtCnt["Select"]+2, currentStmtCnt["Select"]) + require.Equal(t, originStmtCnt["Prepare"]+2, currentStmtCnt["Prepare"]) + require.Equal(t, originStmtCnt["Execute"]+2, currentStmtCnt["Execute"]) + require.Equal(t, originStmtCnt["Replace"]+1, currentStmtCnt["Replace"]) }) } -func (cli *testServerClient) runTestTLSConnection(t *C, overrider configOverrider) error { +func (cli *testServerClient) runTestTLSConnection(t *testing.T, overrider configOverrider) error { dsn := cli.getDSN(overrider) db, err := sql.Open("mysql", dsn) - t.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := db.Close() - t.Assert(err, IsNil) + require.NoError(t, err) }() _, err = db.Exec("USE test") if err != nil { @@ -1873,12 +1830,12 @@ func (cli *testServerClient) runTestTLSConnection(t *C, overrider configOverride return err } -func (cli *testServerClient) runReloadTLS(t *C, overrider configOverrider, errorNoRollback bool) error { +func (cli *testServerClient) runReloadTLS(t *testing.T, overrider configOverrider, errorNoRollback bool) error { db, err := sql.Open("mysql", cli.getDSN(overrider)) - t.Assert(err, IsNil) + require.NoError(t, err) defer func() { err := db.Close() - t.Assert(err, IsNil) + require.NoError(t, err) }() sql := "alter instance reload tls" if errorNoRollback { @@ -1888,35 +1845,35 @@ func (cli *testServerClient) runReloadTLS(t *C, overrider configOverrider, error return err } -func (cli *testServerClient) runTestSumAvg(c *C) { - cli.runTests(c, nil, func(dbt *DBTest) { - dbt.mustExec("create table sumavg (a int, b decimal, c double)") - dbt.mustExec("insert sumavg values (1, 1, 1)") - rows := dbt.mustQuery("select sum(a), sum(b), sum(c) from sumavg") - c.Assert(rows.Next(), IsTrue) +func (cli *testServerClient) runTestSumAvg(t *testing.T) { + cli.runTests(t, nil, func(dbt *testkit.DBTestKit) { + dbt.MustExec("create table sumavg (a int, b decimal, c double)") + dbt.MustExec("insert sumavg values (1, 1, 1)") + rows := dbt.MustQuery("select sum(a), sum(b), sum(c) from sumavg") + require.True(t, rows.Next()) var outA, outB, outC float64 err := rows.Scan(&outA, &outB, &outC) - c.Assert(err, IsNil) - c.Assert(outA, Equals, 1.0) - c.Assert(outB, Equals, 1.0) - c.Assert(outC, Equals, 1.0) - rows = dbt.mustQuery("select avg(a), avg(b), avg(c) from sumavg") - c.Assert(rows.Next(), IsTrue) + require.NoError(t, err) + require.Equal(t, 1.0, outA) + require.Equal(t, 1.0, outB) + require.Equal(t, 1.0, outC) + rows = dbt.MustQuery("select avg(a), avg(b), avg(c) from sumavg") + require.True(t, rows.Next()) err = rows.Scan(&outA, &outB, &outC) - c.Assert(err, IsNil) - c.Assert(outA, Equals, 1.0) - c.Assert(outB, Equals, 1.0) - c.Assert(outC, Equals, 1.0) + require.NoError(t, err) + require.Equal(t, 1.0, outA) + require.Equal(t, 1.0, outB) + require.Equal(t, 1.0, outC) }) } -func (cli *testServerClient) getMetrics(t *C) []byte { +func (cli *testServerClient) getMetrics(t *testing.T) []byte { resp, err := cli.fetchStatus("/metrics") - t.Assert(err, IsNil) + require.NoError(t, err) content, err := io.ReadAll(resp.Body) - t.Assert(err, IsNil) + require.NoError(t, err) err = resp.Body.Close() - t.Assert(err, IsNil) + require.NoError(t, err) return content } @@ -1972,65 +1929,65 @@ func (cli *testServerClient) waitUntilServerOnline() { } } -func (cli *testServerClient) runTestInitConnect(c *C) { +func (cli *testServerClient) runTestInitConnect(t *testing.T) { - cli.runTests(c, nil, func(dbt *DBTest) { - dbt.mustExec(`SET GLOBAL init_connect="insert into test.ts VALUES (NOW());SET @a=1;"`) - dbt.mustExec(`CREATE USER init_nonsuper`) - dbt.mustExec(`CREATE USER init_super`) - dbt.mustExec(`GRANT SELECT, INSERT, DROP ON test.* TO init_nonsuper`) - dbt.mustExec(`GRANT SELECT, INSERT, DROP, SUPER ON *.* TO init_super`) - dbt.mustExec(`CREATE TABLE ts (a TIMESTAMP)`) + cli.runTests(t, nil, func(dbt *testkit.DBTestKit) { + dbt.MustExec(`SET GLOBAL init_connect="insert into test.ts VALUES (NOW());SET @a=1;"`) + dbt.MustExec(`CREATE USER init_nonsuper`) + dbt.MustExec(`CREATE USER init_super`) + dbt.MustExec(`GRANT SELECT, INSERT, DROP ON test.* TO init_nonsuper`) + dbt.MustExec(`GRANT SELECT, INSERT, DROP, SUPER ON *.* TO init_super`) + dbt.MustExec(`CREATE TABLE ts (a TIMESTAMP)`) }) // test init_nonsuper - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.User = "init_nonsuper" - }, func(dbt *DBTest) { - rows := dbt.mustQuery(`SELECT @a`) - c.Assert(rows.Next(), IsTrue) + }, func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery(`SELECT @a`) + require.True(t, rows.Next()) var a int err := rows.Scan(&a) - c.Assert(err, IsNil) - dbt.Check(a, Equals, 1) - c.Assert(rows.Close(), IsNil) + require.NoError(t, err) + require.Equal(t, 1, a) + require.NoError(t, rows.Close()) }) // test init_super - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.User = "init_super" - }, func(dbt *DBTest) { - rows := dbt.mustQuery(`SELECT IFNULL(@a,"")`) - c.Assert(rows.Next(), IsTrue) + }, func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery(`SELECT IFNULL(@a,"")`) + require.True(t, rows.Next()) var a string err := rows.Scan(&a) - c.Assert(err, IsNil) - dbt.Check(a, Equals, "") // null - c.Assert(rows.Close(), IsNil) + require.NoError(t, err) + require.Equal(t, "", a) + require.NoError(t, rows.Close()) // change the init-connect to invalid. - dbt.mustExec(`SET GLOBAL init_connect="invalidstring"`) + dbt.MustExec(`SET GLOBAL init_connect="invalidstring"`) }) // set global init_connect to empty to avoid fail other tests - defer cli.runTests(c, func(config *mysql.Config) { + defer cli.runTests(t, func(config *mysql.Config) { config.User = "init_super" - }, func(dbt *DBTest) { + }, func(dbt *testkit.DBTestKit) { // set init_connect to empty to avoid fail other tests - dbt.mustExec(`SET GLOBAL init_connect=""`) + dbt.MustExec(`SET GLOBAL init_connect=""`) }) db, err := sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { config.User = "init_nonsuper" })) - c.Assert(err, IsNil, Commentf("Error connecting")) // doesn't fail because of lazy loading - defer db.Close() // may already be closed - _, err = db.Exec("SELECT 1") // fails because of init sql - c.Assert(err, NotNil) + require.NoErrorf(t, err, "Error connecting") // doesn't fail because of lazy loading + defer db.Close() // may already be closed + _, err = db.Exec("SELECT 1") // fails because of init sql + require.Error(t, err) } // Client errors are only incremented when using the TiDB Server protocol, // and not internal SQL statements. Thus, this test is in the server-test suite. -func (cli *testServerClient) runTestInfoschemaClientErrors(t *C) { - cli.runTestsOnNewDB(t, nil, "clientErrors", func(dbt *DBTest) { +func (cli *testServerClient) runTestInfoschemaClientErrors(t *testing.T) { + cli.runTestsOnNewDB(t, nil, "clientErrors", func(dbt *testkit.DBTestKit) { clientErrors := []struct { stmt string @@ -2062,7 +2019,7 @@ func (cli *testServerClient) runTestInfoschemaClientErrors(t *C) { for _, tbl := range sources { var errors, warnings int - rows := dbt.mustQuery("SELECT SUM(error_count), SUM(warning_count) FROM information_schema."+tbl+" WHERE error_number = ? GROUP BY error_number", test.errCode) + rows := dbt.MustQuery("SELECT SUM(error_count), SUM(warning_count) FROM information_schema."+tbl+" WHERE error_number = ? GROUP BY error_number", test.errCode) if rows.Next() { rows.Scan(&errors, &warnings) } @@ -2075,7 +2032,7 @@ func (cli *testServerClient) runTestInfoschemaClientErrors(t *C) { warnings++ } var err error - rows, err = dbt.db.Query(test.stmt) + rows, err = dbt.GetDB().Query(test.stmt) if err == nil { // make sure to read the result since the error/warnings are populated in the network send code. if rows.Next() { @@ -2085,13 +2042,13 @@ func (cli *testServerClient) runTestInfoschemaClientErrors(t *C) { rows.Close() } var newErrors, newWarnings int - rows = dbt.mustQuery("SELECT SUM(error_count), SUM(warning_count) FROM information_schema."+tbl+" WHERE error_number = ? GROUP BY error_number", test.errCode) + rows = dbt.MustQuery("SELECT SUM(error_count), SUM(warning_count) FROM information_schema."+tbl+" WHERE error_number = ? GROUP BY error_number", test.errCode) if rows.Next() { rows.Scan(&newErrors, &newWarnings) } rows.Close() - dbt.Check(newErrors, Equals, errors) - dbt.Check(newWarnings, Equals, warnings, Commentf("source=information_schema.%s code=%d statement=%s", tbl, test.errCode, test.stmt)) + require.Equal(t, errors, newErrors) + require.Equalf(t, warnings, newWarnings, "source=information_schema.%s code=%d statement=%s", tbl, test.errCode, test.stmt) } } diff --git a/server/tidb_serial_test.go b/server/tidb_serial_test.go new file mode 100644 index 0000000000000..0431baa32fa8f --- /dev/null +++ b/server/tidb_serial_test.go @@ -0,0 +1,499 @@ +// 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 server + +import ( + "context" + "crypto/x509" + "os" + "sync/atomic" + "testing" + "time" + + "github.com/go-sql-driver/mysql" + "github.com/pingcap/errors" + "github.com/pingcap/tidb/config" + tmysql "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/collate" + "github.com/stretchr/testify/require" +) + +// this test will change `kv.TxnTotalSizeLimit` which may affect other test suites, +// so we must make it running in serial. +func TestLoadData(t *testing.T) { + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestLoadData(t, ts.server) + ts.runTestLoadDataWithSelectIntoOutfile(t, ts.server) + ts.runTestLoadDataForSlowLog(t, ts.server) +} + +func TestConfigDefaultValue(t *testing.T) { + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestsOnNewDB(t, nil, "config", func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select @@tidb_slow_log_threshold;") + ts.checkRows(t, rows, "300") + }) +} + +// Fix issue#22540. Change tidb_dml_batch_size, +// then check if load data into table with auto random column works properly. +func TestLoadDataAutoRandom(t *testing.T) { + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestLoadDataAutoRandom(t) +} + +func TestLoadDataAutoRandomWithSpecialTerm(t *testing.T) { + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestLoadDataAutoRandomWithSpecialTerm(t) +} + +func TestExplainFor(t *testing.T) { + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestExplainForConn(t) +} + +func TestStmtCount(t *testing.T) { + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestStmtCount(t) +} + +func TestLoadDataListPartition(t *testing.T) { + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestLoadDataForListPartition(t) + ts.runTestLoadDataForListPartition2(t) + ts.runTestLoadDataForListColumnPartition(t) + ts.runTestLoadDataForListColumnPartition2(t) +} + +func TestTLSAuto(t *testing.T) { + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + // Start the server without TLS configure, letting the server create these as AutoTLS is enabled + connOverrider := func(config *mysql.Config) { + config.TLSConfig = "skip-verify" + } + cli := newTestServerClient() + cfg := newTestConfig() + cfg.Socket = "" + cfg.Port = cli.port + cfg.Status.ReportStatus = false + cfg.Security.AutoTLS = true + cfg.Security.RSAKeySize = 528 // Reduces unittest runtime + err := os.MkdirAll(cfg.TempStoragePath, 0700) + require.NoError(t, err) + server, err := NewServer(cfg, ts.tidbdrv) + require.NoError(t, err) + cli.port = getPortFromTCPAddr(server.listener.Addr()) + go func() { + err := server.Run() + require.NoError(t, err) + }() + time.Sleep(time.Millisecond * 100) + err = cli.runTestTLSConnection(t, connOverrider) // Relying on automatically created TLS certificates + require.NoError(t, err) + + server.Close() +} + +func TestTLSBasic(t *testing.T) { + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + // Generate valid TLS certificates. + caCert, caKey, err := generateCert(0, "TiDB CA", nil, nil, "/tmp/ca-key.pem", "/tmp/ca-cert.pem") + require.NoError(t, err) + serverCert, _, err := generateCert(1, "tidb-server", caCert, caKey, "/tmp/server-key.pem", "/tmp/server-cert.pem") + require.NoError(t, err) + _, _, err = generateCert(2, "SQL Client Certificate", caCert, caKey, "/tmp/client-key.pem", "/tmp/client-cert.pem") + require.NoError(t, err) + err = registerTLSConfig("client-certificate", "/tmp/ca-cert.pem", "/tmp/client-cert.pem", "/tmp/client-key.pem", "tidb-server", true) + require.NoError(t, err) + + defer func() { + err := os.Remove("/tmp/ca-key.pem") + require.NoError(t, err) + err = os.Remove("/tmp/ca-cert.pem") + require.NoError(t, err) + err = os.Remove("/tmp/server-key.pem") + require.NoError(t, err) + err = os.Remove("/tmp/server-cert.pem") + require.NoError(t, err) + err = os.Remove("/tmp/client-key.pem") + require.NoError(t, err) + err = os.Remove("/tmp/client-cert.pem") + require.NoError(t, err) + }() + + // Start the server with TLS but without CA, in this case the server will not verify client's certificate. + connOverrider := func(config *mysql.Config) { + config.TLSConfig = "skip-verify" + } + cli := newTestServerClient() + cfg := newTestConfig() + cfg.Socket = "" + cfg.Port = cli.port + cfg.Status.ReportStatus = false + cfg.Security = config.Security{ + SSLCert: "/tmp/server-cert.pem", + SSLKey: "/tmp/server-key.pem", + } + server, err := NewServer(cfg, ts.tidbdrv) + require.NoError(t, err) + cli.port = getPortFromTCPAddr(server.listener.Addr()) + go func() { + err := server.Run() + require.NoError(t, err) + }() + time.Sleep(time.Millisecond * 100) + err = cli.runTestTLSConnection(t, connOverrider) // We should establish connection successfully. + require.NoError(t, err) + cli.runTestRegression(t, connOverrider, "TLSRegression") + // Perform server verification. + connOverrider = func(config *mysql.Config) { + config.TLSConfig = "client-certificate" + } + err = cli.runTestTLSConnection(t, connOverrider) // We should establish connection successfully. + require.NoError(t, err, "%v", errors.ErrorStack(err)) + cli.runTestRegression(t, connOverrider, "TLSRegression") + + // Test SSL/TLS session vars + var v *variable.SessionVars + stats, err := server.Stats(v) + require.NoError(t, err) + _, hasKey := stats["Ssl_server_not_after"] + require.True(t, hasKey) + _, hasKey = stats["Ssl_server_not_before"] + require.True(t, hasKey) + require.Equal(t, serverCert.NotAfter.Format("Jan _2 15:04:05 2006 MST"), stats["Ssl_server_not_after"]) + require.Equal(t, serverCert.NotBefore.Format("Jan _2 15:04:05 2006 MST"), stats["Ssl_server_not_before"]) + + server.Close() +} + +func TestTLSVerify(t *testing.T) { + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + // Generate valid TLS certificates. + caCert, caKey, err := generateCert(0, "TiDB CA", nil, nil, "/tmp/ca-key.pem", "/tmp/ca-cert.pem") + require.NoError(t, err) + _, _, err = generateCert(1, "tidb-server", caCert, caKey, "/tmp/server-key.pem", "/tmp/server-cert.pem") + require.NoError(t, err) + _, _, err = generateCert(2, "SQL Client Certificate", caCert, caKey, "/tmp/client-key.pem", "/tmp/client-cert.pem") + require.NoError(t, err) + err = registerTLSConfig("client-certificate", "/tmp/ca-cert.pem", "/tmp/client-cert.pem", "/tmp/client-key.pem", "tidb-server", true) + require.NoError(t, err) + + defer func() { + err := os.Remove("/tmp/ca-key.pem") + require.NoError(t, err) + err = os.Remove("/tmp/ca-cert.pem") + require.NoError(t, err) + err = os.Remove("/tmp/server-key.pem") + require.NoError(t, err) + err = os.Remove("/tmp/server-cert.pem") + require.NoError(t, err) + err = os.Remove("/tmp/client-key.pem") + require.NoError(t, err) + err = os.Remove("/tmp/client-cert.pem") + require.NoError(t, err) + }() + + // Start the server with TLS & CA, if the client presents its certificate, the certificate will be verified. + cli := newTestServerClient() + cfg := newTestConfig() + cfg.Socket = "" + cfg.Port = cli.port + cfg.Status.ReportStatus = false + cfg.Security = config.Security{ + SSLCA: "/tmp/ca-cert.pem", + SSLCert: "/tmp/server-cert.pem", + SSLKey: "/tmp/server-key.pem", + } + server, err := NewServer(cfg, ts.tidbdrv) + require.NoError(t, err) + cli.port = getPortFromTCPAddr(server.listener.Addr()) + go func() { + err := server.Run() + require.NoError(t, err) + }() + time.Sleep(time.Millisecond * 100) + // The client does not provide a certificate, the connection should succeed. + err = cli.runTestTLSConnection(t, nil) + require.NoError(t, err) + connOverrider := func(config *mysql.Config) { + config.TLSConfig = "client-certificate" + } + cli.runTestRegression(t, connOverrider, "TLSRegression") + // The client provides a valid certificate. + connOverrider = func(config *mysql.Config) { + config.TLSConfig = "client-certificate" + } + err = cli.runTestTLSConnection(t, connOverrider) + require.NoError(t, err) + cli.runTestRegression(t, connOverrider, "TLSRegression") + server.Close() + + require.False(t, util.IsTLSExpiredError(errors.New("unknown test"))) + require.False(t, util.IsTLSExpiredError(x509.CertificateInvalidError{Reason: x509.CANotAuthorizedForThisName})) + require.True(t, util.IsTLSExpiredError(x509.CertificateInvalidError{Reason: x509.Expired})) + + _, _, err = util.LoadTLSCertificates("", "wrong key", "wrong cert", true, 528) + require.Error(t, err) + _, _, err = util.LoadTLSCertificates("wrong ca", "/tmp/server-key.pem", "/tmp/server-cert.pem", true, 528) + require.Error(t, err) +} + +func TestErrorNoRollback(t *testing.T) { + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + // Generate valid TLS certificates. + caCert, caKey, err := generateCert(0, "TiDB CA", nil, nil, "/tmp/ca-key-rollback.pem", "/tmp/ca-cert-rollback.pem") + require.NoError(t, err) + _, _, err = generateCert(1, "tidb-server", caCert, caKey, "/tmp/server-key-rollback.pem", "/tmp/server-cert-rollback.pem") + require.NoError(t, err) + _, _, err = generateCert(2, "SQL Client Certificate", caCert, caKey, "/tmp/client-key-rollback.pem", "/tmp/client-cert-rollback.pem") + require.NoError(t, err) + err = registerTLSConfig("client-cert-rollback-test", "/tmp/ca-cert-rollback.pem", "/tmp/client-cert-rollback.pem", "/tmp/client-key-rollback.pem", "tidb-server", true) + require.NoError(t, err) + + defer func() { + os.Remove("/tmp/ca-key-rollback.pem") + os.Remove("/tmp/ca-cert-rollback.pem") + + os.Remove("/tmp/server-key-rollback.pem") + os.Remove("/tmp/server-cert-rollback.pem") + os.Remove("/tmp/client-key-rollback.pem") + os.Remove("/tmp/client-cert-rollback.pem") + }() + + cli := newTestServerClient() + cfg := newTestConfig() + cfg.Socket = "" + cfg.Port = cli.port + cfg.Status.ReportStatus = false + + cfg.Security = config.Security{ + RequireSecureTransport: true, + SSLCA: "wrong path", + SSLCert: "wrong path", + SSLKey: "wrong path", + } + _, err = NewServer(cfg, ts.tidbdrv) + require.Error(t, err) + + // test reload tls fail with/without "error no rollback option" + cfg.Security = config.Security{ + SSLCA: "/tmp/ca-cert-rollback.pem", + SSLCert: "/tmp/server-cert-rollback.pem", + SSLKey: "/tmp/server-key-rollback.pem", + } + server, err := NewServer(cfg, ts.tidbdrv) + require.NoError(t, err) + cli.port = getPortFromTCPAddr(server.listener.Addr()) + go func() { + err := server.Run() + require.NoError(t, err) + }() + defer server.Close() + time.Sleep(time.Millisecond * 100) + connOverrider := func(config *mysql.Config) { + config.TLSConfig = "client-cert-rollback-test" + } + err = cli.runTestTLSConnection(t, connOverrider) + require.NoError(t, err) + os.Remove("/tmp/server-key-rollback.pem") + err = cli.runReloadTLS(t, connOverrider, false) + require.Error(t, err) + tlsCfg := server.getTLSConfig() + require.NotNil(t, tlsCfg) + err = cli.runReloadTLS(t, connOverrider, true) + require.NoError(t, err) + tlsCfg = server.getTLSConfig() + require.Nil(t, tlsCfg) +} + +func TestPrepareCount(t *testing.T) { + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + qctx, err := ts.tidbdrv.OpenCtx(uint64(0), 0, uint8(tmysql.DefaultCollationID), "test", nil) + require.NoError(t, err) + prepareCnt := atomic.LoadInt64(&variable.PreparedStmtCount) + ctx := context.Background() + _, err = Execute(ctx, qctx, "use test;") + require.NoError(t, err) + _, err = Execute(ctx, qctx, "drop table if exists t1") + require.NoError(t, err) + _, err = Execute(ctx, qctx, "create table t1 (id int)") + require.NoError(t, err) + stmt, _, _, err := qctx.Prepare("insert into t1 values (?)") + require.NoError(t, err) + require.Equal(t, prepareCnt+1, atomic.LoadInt64(&variable.PreparedStmtCount)) + require.NoError(t, err) + err = qctx.GetStatement(stmt.ID()).Close() + require.NoError(t, err) + require.Equal(t, prepareCnt, atomic.LoadInt64(&variable.PreparedStmtCount)) +} + +func TestDefaultCharacterAndCollation(t *testing.T) { + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + // issue #21194 + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + // 255 is the collation id of mysql client 8 default collation_connection + qctx, err := ts.tidbdrv.OpenCtx(uint64(0), 0, uint8(255), "test", nil) + require.NoError(t, err) + testCase := []struct { + variable string + except string + }{ + {"collation_connection", "utf8mb4_bin"}, + {"character_set_connection", "utf8mb4"}, + {"character_set_client", "utf8mb4"}, + } + + for _, tc := range testCase { + sVars, b := qctx.GetSessionVars().GetSystemVar(tc.variable) + require.True(t, b) + require.Equal(t, tc.except, sVars) + } +} + +func TestReloadTLS(t *testing.T) { + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + // Generate valid TLS certificates. + caCert, caKey, err := generateCert(0, "TiDB CA", nil, nil, "/tmp/ca-key-reload.pem", "/tmp/ca-cert-reload.pem") + require.NoError(t, err) + _, _, err = generateCert(1, "tidb-server", caCert, caKey, "/tmp/server-key-reload.pem", "/tmp/server-cert-reload.pem") + require.NoError(t, err) + _, _, err = generateCert(2, "SQL Client Certificate", caCert, caKey, "/tmp/client-key-reload.pem", "/tmp/client-cert-reload.pem") + require.NoError(t, err) + err = registerTLSConfig("client-certificate-reload", "/tmp/ca-cert-reload.pem", "/tmp/client-cert-reload.pem", "/tmp/client-key-reload.pem", "tidb-server", true) + require.NoError(t, err) + + defer func() { + os.Remove("/tmp/ca-key-reload.pem") + os.Remove("/tmp/ca-cert-reload.pem") + + os.Remove("/tmp/server-key-reload.pem") + os.Remove("/tmp/server-cert-reload.pem") + os.Remove("/tmp/client-key-reload.pem") + os.Remove("/tmp/client-cert-reload.pem") + }() + + // try old cert used in startup configuration. + cli := newTestServerClient() + cfg := newTestConfig() + cfg.Socket = "" + cfg.Port = cli.port + cfg.Status.ReportStatus = false + cfg.Security = config.Security{ + SSLCA: "/tmp/ca-cert-reload.pem", + SSLCert: "/tmp/server-cert-reload.pem", + SSLKey: "/tmp/server-key-reload.pem", + } + server, err := NewServer(cfg, ts.tidbdrv) + require.NoError(t, err) + cli.port = getPortFromTCPAddr(server.listener.Addr()) + go func() { + err := server.Run() + require.NoError(t, err) + }() + time.Sleep(time.Millisecond * 100) + // The client provides a valid certificate. + connOverrider := func(config *mysql.Config) { + config.TLSConfig = "client-certificate-reload" + } + err = cli.runTestTLSConnection(t, connOverrider) + require.NoError(t, err) + + // try reload a valid cert. + tlsCfg := server.getTLSConfig() + cert, err := x509.ParseCertificate(tlsCfg.Certificates[0].Certificate[0]) + require.NoError(t, err) + oldExpireTime := cert.NotAfter + _, _, err = generateCert(1, "tidb-server", caCert, caKey, "/tmp/server-key-reload2.pem", "/tmp/server-cert-reload2.pem", func(c *x509.Certificate) { + c.NotBefore = time.Now().Add(-24 * time.Hour).UTC() + c.NotAfter = time.Now().Add(1 * time.Hour).UTC() + }) + require.NoError(t, err) + err = os.Rename("/tmp/server-key-reload2.pem", "/tmp/server-key-reload.pem") + require.NoError(t, err) + err = os.Rename("/tmp/server-cert-reload2.pem", "/tmp/server-cert-reload.pem") + require.NoError(t, err) + connOverrider = func(config *mysql.Config) { + config.TLSConfig = "skip-verify" + } + err = cli.runReloadTLS(t, connOverrider, false) + require.NoError(t, err) + connOverrider = func(config *mysql.Config) { + config.TLSConfig = "client-certificate-reload" + } + err = cli.runTestTLSConnection(t, connOverrider) + require.NoError(t, err) + + tlsCfg = server.getTLSConfig() + cert, err = x509.ParseCertificate(tlsCfg.Certificates[0].Certificate[0]) + require.NoError(t, err) + newExpireTime := cert.NotAfter + require.True(t, newExpireTime.After(oldExpireTime)) + + // try reload a expired cert. + _, _, err = generateCert(1, "tidb-server", caCert, caKey, "/tmp/server-key-reload3.pem", "/tmp/server-cert-reload3.pem", func(c *x509.Certificate) { + c.NotBefore = time.Now().Add(-24 * time.Hour).UTC() + c.NotAfter = c.NotBefore.Add(1 * time.Hour).UTC() + }) + require.NoError(t, err) + err = os.Rename("/tmp/server-key-reload3.pem", "/tmp/server-key-reload.pem") + require.NoError(t, err) + err = os.Rename("/tmp/server-cert-reload3.pem", "/tmp/server-cert-reload.pem") + require.NoError(t, err) + connOverrider = func(config *mysql.Config) { + config.TLSConfig = "skip-verify" + } + err = cli.runReloadTLS(t, connOverrider, false) + require.NoError(t, err) + connOverrider = func(config *mysql.Config) { + config.TLSConfig = "client-certificate-reload" + } + err = cli.runTestTLSConnection(t, connOverrider) + require.NotNil(t, err) + require.Truef(t, util.IsTLSExpiredError(err), "real error is %+v", err) + server.Close() +} diff --git a/server/tidb_test.go b/server/tidb_test.go index b9f9ddbf9f9e3..01a19d70df6d1 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -32,46 +32,31 @@ import ( "os" "path/filepath" "strings" - "sync/atomic" + "testing" "time" "github.com/go-sql-driver/mysql" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser" tmysql "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/plancodec" - "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/topsql/reporter" mockTopSQLReporter "github.com/pingcap/tidb/util/topsql/reporter/mock" "github.com/pingcap/tidb/util/topsql/tracecpu" mockTopSQLTraceCPU "github.com/pingcap/tidb/util/topsql/tracecpu/mock" + "github.com/stretchr/testify/require" ) type tidbTestSuite struct { - *tidbTestSuiteBase -} - -type tidbTestSerialSuite struct { - *tidbTestSuiteBase -} - -type tidbTestTopSQLSuite struct { - *tidbTestSuiteBase -} - -type tidbTestSuiteBase struct { *testServerClient tidbdrv *TiDBDriver server *Server @@ -79,55 +64,16 @@ type tidbTestSuiteBase struct { store kv.Storage } -func newTiDBTestSuiteBase() *tidbTestSuiteBase { - return &tidbTestSuiteBase{ - testServerClient: newTestServerClient(), - } -} - -var _ = Suite(&tidbTestSuite{newTiDBTestSuiteBase()}) -var _ = SerialSuites(&tidbTestSerialSuite{newTiDBTestSuiteBase()}) -var _ = SerialSuites(&tidbTestTopSQLSuite{newTiDBTestSuiteBase()}) - -func (ts *tidbTestSuite) SetUpSuite(c *C) { - metrics.RegisterMetrics() - ts.tidbTestSuiteBase.SetUpSuite(c) -} - -func (ts *tidbTestSuite) TearDownSuite(c *C) { - ts.tidbTestSuiteBase.TearDownSuite(c) -} - -func (ts *tidbTestTopSQLSuite) SetUpSuite(c *C) { - ts.tidbTestSuiteBase.SetUpSuite(c) - - // Initialize global variable for top-sql test. - db, err := sql.Open("mysql", ts.getDSN()) - c.Assert(err, IsNil, Commentf("Error connecting")) - defer func() { - err := db.Close() - c.Assert(err, IsNil) - }() - - dbt := &DBTest{c, db} - dbt.mustExec("set @@global.tidb_top_sql_precision_seconds=1;") - dbt.mustExec("set @@global.tidb_top_sql_report_interval_seconds=2;") - dbt.mustExec("set @@global.tidb_top_sql_max_statement_count=5;") - - tracecpu.GlobalSQLCPUProfiler.Run() -} - -func (ts *tidbTestTopSQLSuite) TearDownSuite(c *C) { - ts.tidbTestSuiteBase.TearDownSuite(c) -} +func createTidbTestSuite(t *testing.T) (*tidbTestSuite, func()) { + ts := &tidbTestSuite{testServerClient: newTestServerClient()} -func (ts *tidbTestSuiteBase) SetUpSuite(c *C) { + // setup tidbTestSuite var err error ts.store, err = mockstore.NewMockStore() session.DisableStats4Test() - c.Assert(err, IsNil) + require.NoError(t, err) ts.domain, err = session.BootstrapSession(ts.store) - c.Assert(err, IsNil) + require.NoError(t, err) ts.tidbdrv = NewTiDBDriver(ts.store) cfg := newTestConfig() cfg.Socket = "" @@ -136,145 +82,165 @@ func (ts *tidbTestSuiteBase) SetUpSuite(c *C) { cfg.Status.StatusPort = ts.statusPort cfg.Performance.TCPKeepAlive = true err = logutil.InitLogger(cfg.Log.ToLogConfig()) - c.Assert(err, IsNil) + require.NoError(t, err) server, err := NewServer(cfg, ts.tidbdrv) - c.Assert(err, IsNil) + require.NoError(t, err) ts.port = getPortFromTCPAddr(server.listener.Addr()) ts.statusPort = getPortFromTCPAddr(server.statusListener.Addr()) ts.server = server go func() { err := ts.server.Run() - c.Assert(err, IsNil) + require.NoError(t, err) }() ts.waitUntilServerOnline() -} -func (ts *tidbTestSuiteBase) TearDownSuite(c *C) { - if ts.store != nil { - ts.store.Close() - } - if ts.domain != nil { - ts.domain.Close() - } - if ts.server != nil { - ts.server.Close() + cleanup := func() { + if ts.store != nil { + ts.store.Close() + } + if ts.domain != nil { + ts.domain.Close() + } + if ts.server != nil { + ts.server.Close() + } } -} -func (ts *tidbTestSuite) TestRegression(c *C) { - if regression { - c.Parallel() - ts.runTestRegression(c, nil, "Regression") - } + return ts, cleanup } -func (ts *tidbTestSuite) TestUint64(c *C) { - ts.runTestPrepareResultFieldType(c) +type tidbTestTopSQLSuite struct { + *tidbTestSuite } -func (ts *tidbTestSuite) TestSpecialType(c *C) { - c.Parallel() - ts.runTestSpecialType(c) -} +func createTidbTestTopSQLSuite(t *testing.T) (*tidbTestTopSQLSuite, func()) { + base, cleanup := createTidbTestSuite(t) -func (ts *tidbTestSuite) TestPreparedString(c *C) { - c.Parallel() - ts.runTestPreparedString(c) -} + ts := &tidbTestTopSQLSuite{base} -func (ts *tidbTestSuite) TestPreparedTimestamp(c *C) { - c.Parallel() - ts.runTestPreparedTimestamp(c) -} + // Initialize global variable for top-sql test. + db, err := sql.Open("mysql", ts.getDSN()) + require.NoErrorf(t, err, "Error connecting") + defer func() { + err := db.Close() + require.NoError(t, err) + }() -func (ts *tidbTestSerialSuite) TestConfigDefaultValue(c *C) { - ts.runTestsOnNewDB(c, nil, "config", func(dbt *DBTest) { - rows := dbt.mustQuery("select @@tidb_slow_log_threshold;") - ts.checkRows(c, rows, "300") - }) -} + dbt := testkit.NewDBTestKit(t, db) + dbt.MustExec("set @@global.tidb_top_sql_precision_seconds=1;") + dbt.MustExec("set @@global.tidb_top_sql_report_interval_seconds=2;") + dbt.MustExec("set @@global.tidb_top_sql_max_statement_count=5;") + + tracecpu.GlobalSQLCPUProfiler.Run() -// this test will change `kv.TxnTotalSizeLimit` which may affect other test suites, -// so we must make it running in serial. -func (ts *tidbTestSerialSuite) TestLoadData(c *C) { - ts.runTestLoadData(c, ts.server) - ts.runTestLoadDataWithSelectIntoOutfile(c, ts.server) - ts.runTestLoadDataForSlowLog(c, ts.server) + return ts, cleanup } -func (ts *tidbTestSerialSuite) TestLoadDataListPartition(c *C) { - ts.runTestLoadDataForListPartition(c) - ts.runTestLoadDataForListPartition2(c) - ts.runTestLoadDataForListColumnPartition(c) - ts.runTestLoadDataForListColumnPartition2(c) +func TestRegression(t *testing.T) { + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + if regression { + t.Parallel() + ts.runTestRegression(t, nil, "Regression") + } } -// Fix issue#22540. Change tidb_dml_batch_size, -// then check if load data into table with auto random column works properly. -func (ts *tidbTestSerialSuite) TestLoadDataAutoRandom(c *C) { - ts.runTestLoadDataAutoRandom(c) +func TestUint64(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestPrepareResultFieldType(t) } -func (ts *tidbTestSerialSuite) TestLoadDataAutoRandomWithSpecialTerm(c *C) { - ts.runTestLoadDataAutoRandomWithSpecialTerm(c) +func TestSpecialType(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestSpecialType(t) } -func (ts *tidbTestSerialSuite) TestExplainFor(c *C) { - ts.runTestExplainForConn(c) +func TestPreparedString(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestPreparedString(t) } -func (ts *tidbTestSerialSuite) TestStmtCount(c *C) { - ts.runTestStmtCount(c) +func TestPreparedTimestamp(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestPreparedTimestamp(t) } -func (ts *tidbTestSuite) TestConcurrentUpdate(c *C) { - c.Parallel() - ts.runTestConcurrentUpdate(c) +func TestConcurrentUpdate(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestConcurrentUpdate(t) } -func (ts *tidbTestSuite) TestErrorCode(c *C) { - c.Parallel() - ts.runTestErrorCode(c) +func TestErrorCode(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestErrorCode(t) } -func (ts *tidbTestSuite) TestAuth(c *C) { - c.Parallel() - ts.runTestAuth(c) - ts.runTestIssue3682(c) +func TestAuth(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestAuth(t) + ts.runTestIssue3682(t) } -func (ts *tidbTestSuite) TestIssues(c *C) { - c.Parallel() - ts.runTestIssue3662(c) - ts.runTestIssue3680(c) - ts.runTestIssue22646(c) +func TestIssues(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestIssue3662(t) + ts.runTestIssue3680(t) + ts.runTestIssue22646(t) } -func (ts *tidbTestSuite) TestDBNameEscape(c *C) { - c.Parallel() - ts.runTestDBNameEscape(c) +func TestDBNameEscape(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + ts.runTestDBNameEscape(t) } -func (ts *tidbTestSuite) TestResultFieldTableIsNull(c *C) { - c.Parallel() - ts.runTestResultFieldTableIsNull(c) +func TestResultFieldTableIsNull(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestResultFieldTableIsNull(t) } -func (ts *tidbTestSuite) TestStatusAPI(c *C) { - c.Parallel() - ts.runTestStatusAPI(c) +func TestStatusAPI(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runTestStatusAPI(t) } -func (ts *tidbTestSuite) TestStatusPort(c *C) { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - defer store.Close() - session.DisableStats4Test() - dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - defer dom.Close() - ts.tidbdrv = NewTiDBDriver(store) +func TestStatusPort(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + cfg := newTestConfig() cfg.Socket = "" cfg.Port = 0 @@ -283,15 +249,19 @@ func (ts *tidbTestSuite) TestStatusPort(c *C) { cfg.Performance.TCPKeepAlive = true server, err := NewServer(cfg, ts.tidbdrv) - c.Assert(err, NotNil) - c.Assert(server, IsNil) + require.Error(t, err) + require.Nil(t, server) } -func (ts *tidbTestSuite) TestStatusAPIWithTLS(c *C) { +func TestStatusAPIWithTLS(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + caCert, caKey, err := generateCert(0, "TiDB CA 2", nil, nil, "/tmp/ca-key-2.pem", "/tmp/ca-cert-2.pem") - c.Assert(err, IsNil) + require.NoError(t, err) _, _, err = generateCert(1, "tidb-server-2", caCert, caKey, "/tmp/server-key-2.pem", "/tmp/server-cert-2.pem") - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { os.Remove("/tmp/ca-key-2.pem") @@ -310,27 +280,31 @@ func (ts *tidbTestSuite) TestStatusAPIWithTLS(c *C) { cfg.Security.ClusterSSLCert = "/tmp/server-cert-2.pem" cfg.Security.ClusterSSLKey = "/tmp/server-key-2.pem" server, err := NewServer(cfg, ts.tidbdrv) - c.Assert(err, IsNil) + require.NoError(t, err) cli.port = getPortFromTCPAddr(server.listener.Addr()) cli.statusPort = getPortFromTCPAddr(server.statusListener.Addr()) go func() { err := server.Run() - c.Assert(err, IsNil) + require.NoError(t, err) }() time.Sleep(time.Millisecond * 100) // https connection should work. - ts.runTestStatusAPI(c) + ts.runTestStatusAPI(t) // but plain http connection should fail. cli.statusScheme = "http" _, err = cli.fetchStatus("/status") // nolint: bodyclose - c.Assert(err, NotNil) + require.Error(t, err) server.Close() } -func (ts *tidbTestSuite) TestStatusAPIWithTLSCNCheck(c *C) { +func TestStatusAPIWithTLSCNCheck(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + caPath := filepath.Join(os.TempDir(), "ca-cert-cn.pem") serverKeyPath := filepath.Join(os.TempDir(), "server-key-cn.pem") serverCertPath := filepath.Join(os.TempDir(), "server-cert-cn.pem") @@ -340,17 +314,17 @@ func (ts *tidbTestSuite) TestStatusAPIWithTLSCNCheck(c *C) { client2CertPath := filepath.Join(os.TempDir(), "client-cert-cn-check-b.pem") caCert, caKey, err := generateCert(0, "TiDB CA CN CHECK", nil, nil, filepath.Join(os.TempDir(), "ca-key-cn.pem"), caPath) - c.Assert(err, IsNil) + require.NoError(t, err) _, _, err = generateCert(1, "tidb-server-cn-check", caCert, caKey, serverKeyPath, serverCertPath) - c.Assert(err, IsNil) + require.NoError(t, err) _, _, err = generateCert(2, "tidb-client-cn-check-a", caCert, caKey, client1KeyPath, client1CertPath, func(c *x509.Certificate) { c.Subject.CommonName = "tidb-client-1" }) - c.Assert(err, IsNil) + require.NoError(t, err) _, _, err = generateCert(3, "tidb-client-cn-check-b", caCert, caKey, client2KeyPath, client2CertPath, func(c *x509.Certificate) { c.Subject.CommonName = "tidb-client-2" }) - c.Assert(err, IsNil) + require.NoError(t, err) cli := newTestServerClient() cli.statusScheme = "https" @@ -363,37 +337,38 @@ func (ts *tidbTestSuite) TestStatusAPIWithTLSCNCheck(c *C) { cfg.Security.ClusterSSLKey = serverKeyPath cfg.Security.ClusterVerifyCN = []string{"tidb-client-2"} server, err := NewServer(cfg, ts.tidbdrv) - c.Assert(err, IsNil) + require.NoError(t, err) + cli.port = getPortFromTCPAddr(server.listener.Addr()) cli.statusPort = getPortFromTCPAddr(server.statusListener.Addr()) go func() { err := server.Run() - c.Assert(err, IsNil) + require.NoError(t, err) }() defer server.Close() time.Sleep(time.Millisecond * 100) - hc := newTLSHttpClient(c, caPath, + hc := newTLSHttpClient(t, caPath, client1CertPath, client1KeyPath, ) _, err = hc.Get(cli.statusURL("/status")) // nolint: bodyclose - c.Assert(err, NotNil) + require.Error(t, err) - hc = newTLSHttpClient(c, caPath, + hc = newTLSHttpClient(t, caPath, client2CertPath, client2KeyPath, ) resp, err := hc.Get(cli.statusURL("/status")) - c.Assert(err, IsNil) - c.Assert(resp.Body.Close(), IsNil) + require.NoError(t, err) + require.Nil(t, resp.Body.Close()) } -func newTLSHttpClient(c *C, caFile, certFile, keyFile string) *http.Client { +func newTLSHttpClient(t *testing.T, caFile, certFile, keyFile string) *http.Client { cert, err := tls.LoadX509KeyPair(certFile, keyFile) - c.Assert(err, IsNil) + require.NoError(t, err) caCert, err := os.ReadFile(caFile) - c.Assert(err, IsNil) + require.NoError(t, err) caCertPool := x509.NewCertPool() caCertPool.AppendCertsFromPEM(caCert) tlsConfig := &tls.Config{ @@ -405,86 +380,113 @@ func newTLSHttpClient(c *C, caFile, certFile, keyFile string) *http.Client { return &http.Client{Transport: &http.Transport{TLSClientConfig: tlsConfig}} } -func (ts *tidbTestSuite) TestMultiStatements(c *C) { - c.Parallel() - ts.runFailedTestMultiStatements(c) - ts.runTestMultiStatements(c) +func TestMultiStatements(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + ts.runFailedTestMultiStatements(t) + ts.runTestMultiStatements(t) } -func (ts *tidbTestSuite) TestSocketForwarding(c *C) { +func TestSocketForwarding(t *testing.T) { + t.Parallel() + osTempDir := os.TempDir() + tempDir, err := os.MkdirTemp(osTempDir, "tidb-test.*.socket") + require.NoError(t, err) + socketFile := tempDir + "/tidbtest.sock" // Unix Socket does not work on Windows, so '/' should be OK + defer os.RemoveAll(tempDir) + + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + cli := newTestServerClient() cfg := newTestConfig() - cfg.Socket = "/tmp/tidbtest.sock" + cfg.Socket = socketFile cfg.Port = cli.port os.Remove(cfg.Socket) cfg.Status.ReportStatus = false server, err := NewServer(cfg, ts.tidbdrv) - c.Assert(err, IsNil) + require.NoError(t, err) cli.port = getPortFromTCPAddr(server.listener.Addr()) go func() { err := server.Run() - c.Assert(err, IsNil) + require.NoError(t, err) }() time.Sleep(time.Millisecond * 100) defer server.Close() - cli.runTestRegression(c, func(config *mysql.Config) { + cli.runTestRegression(t, func(config *mysql.Config) { config.User = "root" config.Net = "unix" - config.Addr = "/tmp/tidbtest.sock" + config.Addr = socketFile config.DBName = "test" config.Params = map[string]string{"sql_mode": "'STRICT_ALL_TABLES'"} }, "SocketRegression") } -func (ts *tidbTestSuite) TestSocket(c *C) { +func TestSocket(t *testing.T) { + t.Parallel() + osTempDir := os.TempDir() + tempDir, err := os.MkdirTemp(osTempDir, "tidb-test.*.socket") + require.NoError(t, err) + socketFile := tempDir + "/tidbtest.sock" // Unix Socket does not work on Windows, so '/' should be OK + defer os.RemoveAll(tempDir) + cfg := newTestConfig() - cfg.Socket = "/tmp/tidbtest.sock" + cfg.Socket = socketFile cfg.Port = 0 os.Remove(cfg.Socket) cfg.Host = "" cfg.Status.ReportStatus = false + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + server, err := NewServer(cfg, ts.tidbdrv) - c.Assert(err, IsNil) + require.NoError(t, err) go func() { err := server.Run() - c.Assert(err, IsNil) + require.NoError(t, err) }() time.Sleep(time.Millisecond * 100) defer server.Close() // a fake server client, config is override, just used to run tests cli := newTestServerClient() - cli.runTestRegression(c, func(config *mysql.Config) { + cli.runTestRegression(t, func(config *mysql.Config) { config.User = "root" config.Net = "unix" - config.Addr = "/tmp/tidbtest.sock" + config.Addr = socketFile config.DBName = "test" config.Params = map[string]string{"sql_mode": "STRICT_ALL_TABLES"} }, "SocketRegression") - } -func (ts *tidbTestSuite) TestSocketAndIp(c *C) { +func TestSocketAndIp(t *testing.T) { + t.Parallel() osTempDir := os.TempDir() tempDir, err := os.MkdirTemp(osTempDir, "tidb-test.*.socket") - c.Assert(err, IsNil) + require.NoError(t, err) socketFile := tempDir + "/tidbtest.sock" // Unix Socket does not work on Windows, so '/' should be OK defer os.RemoveAll(tempDir) + cli := newTestServerClient() cfg := newTestConfig() cfg.Socket = socketFile cfg.Port = cli.port cfg.Status.ReportStatus = false + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + server, err := NewServer(cfg, ts.tidbdrv) - c.Assert(err, IsNil) + require.NoError(t, err) cli.port = getPortFromTCPAddr(server.listener.Addr()) go func() { err := server.Run() - c.Assert(err, IsNil) + require.NoError(t, err) }() time.Sleep(time.Millisecond * 100) defer server.Close() @@ -492,188 +494,193 @@ func (ts *tidbTestSuite) TestSocketAndIp(c *C) { // Test with Socket connection + Setup user1@% for all host access cli.port = getPortFromTCPAddr(server.listener.Addr()) defer func() { - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.User = "root" }, - func(dbt *DBTest) { - dbt.mustQuery("DROP USER IF EXISTS 'user1'@'%'") - dbt.mustQuery("DROP USER IF EXISTS 'user1'@'localhost'") - dbt.mustQuery("DROP USER IF EXISTS 'user1'@'127.0.0.1'") + func(dbt *testkit.DBTestKit) { + dbt.MustQuery("DROP USER IF EXISTS 'user1'@'%'") + dbt.MustQuery("DROP USER IF EXISTS 'user1'@'localhost'") + dbt.MustQuery("DROP USER IF EXISTS 'user1'@'127.0.0.1'") }) }() - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.User = "root" config.Net = "unix" config.Addr = socketFile config.DBName = "test" }, - func(dbt *DBTest) { - rows := dbt.mustQuery("select user()") - cli.checkRows(c, rows, "root@localhost") - rows = dbt.mustQuery("show grants") - cli.checkRows(c, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") - dbt.mustQuery("CREATE USER user1@'%'") - dbt.mustQuery("GRANT SELECT ON test.* TO user1@'%'") + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") + cli.checkRows(t, rows, "root@localhost") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") + dbt.MustQuery("CREATE USER user1@'%'") + dbt.MustQuery("GRANT SELECT ON test.* TO user1@'%'") }) // Test with Network interface connection with all hosts - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.User = "user1" config.DBName = "test" }, - func(dbt *DBTest) { - rows := dbt.mustQuery("select user()") + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") // NOTICE: this is not compatible with MySQL! (MySQL would report user1@localhost also for 127.0.0.1) - cli.checkRows(c, rows, "user1@127.0.0.1") - rows = dbt.mustQuery("show grants") - cli.checkRows(c, rows, "GRANT USAGE ON *.* TO 'user1'@'%'\nGRANT SELECT ON test.* TO 'user1'@'%'") + cli.checkRows(t, rows, "user1@127.0.0.1") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'user1'@'%'\nGRANT SELECT ON test.* TO 'user1'@'%'") }) // Test with unix domain socket file connection with all hosts - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.Net = "unix" config.Addr = socketFile config.User = "user1" config.DBName = "test" }, - func(dbt *DBTest) { - rows := dbt.mustQuery("select user()") - cli.checkRows(c, rows, "user1@localhost") - rows = dbt.mustQuery("show grants") - cli.checkRows(c, rows, "GRANT USAGE ON *.* TO 'user1'@'%'\nGRANT SELECT ON test.* TO 'user1'@'%'") + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") + cli.checkRows(t, rows, "user1@localhost") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'user1'@'%'\nGRANT SELECT ON test.* TO 'user1'@'%'") }) // Setup user1@127.0.0.1 for loop back network interface access - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.User = "root" config.DBName = "test" }, - func(dbt *DBTest) { - rows := dbt.mustQuery("select user()") + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") // NOTICE: this is not compatible with MySQL! (MySQL would report user1@localhost also for 127.0.0.1) - cli.checkRows(c, rows, "root@127.0.0.1") - rows = dbt.mustQuery("show grants") - cli.checkRows(c, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") - dbt.mustQuery("CREATE USER user1@127.0.0.1") - dbt.mustQuery("GRANT SELECT,INSERT ON test.* TO user1@'127.0.0.1'") + cli.checkRows(t, rows, "root@127.0.0.1") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") + dbt.MustQuery("CREATE USER user1@127.0.0.1") + dbt.MustQuery("GRANT SELECT,INSERT ON test.* TO user1@'127.0.0.1'") }) // Test with Network interface connection with all hosts - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.User = "user1" config.DBName = "test" }, - func(dbt *DBTest) { - rows := dbt.mustQuery("select user()") + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") // NOTICE: this is not compatible with MySQL! (MySQL would report user1@localhost also for 127.0.0.1) - cli.checkRows(c, rows, "user1@127.0.0.1") - rows = dbt.mustQuery("show grants") - cli.checkRows(c, rows, "GRANT USAGE ON *.* TO 'user1'@'127.0.0.1'\nGRANT SELECT,INSERT ON test.* TO 'user1'@'127.0.0.1'") + cli.checkRows(t, rows, "user1@127.0.0.1") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'user1'@'127.0.0.1'\nGRANT SELECT,INSERT ON test.* TO 'user1'@'127.0.0.1'") }) // Test with unix domain socket file connection with all hosts - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.Net = "unix" config.Addr = socketFile config.User = "user1" config.DBName = "test" }, - func(dbt *DBTest) { - rows := dbt.mustQuery("select user()") - cli.checkRows(c, rows, "user1@localhost") - rows = dbt.mustQuery("show grants") - cli.checkRows(c, rows, "GRANT USAGE ON *.* TO 'user1'@'%'\nGRANT SELECT ON test.* TO 'user1'@'%'") + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") + cli.checkRows(t, rows, "user1@localhost") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'user1'@'%'\nGRANT SELECT ON test.* TO 'user1'@'%'") }) // Setup user1@localhost for socket (and if MySQL compatible; loop back network interface access) - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.Net = "unix" config.Addr = socketFile config.User = "root" config.DBName = "test" }, - func(dbt *DBTest) { - rows := dbt.mustQuery("select user()") - cli.checkRows(c, rows, "root@localhost") - rows = dbt.mustQuery("show grants") - cli.checkRows(c, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") - dbt.mustQuery("CREATE USER user1@localhost") - dbt.mustQuery("GRANT SELECT,INSERT,UPDATE,DELETE ON test.* TO user1@localhost") + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") + cli.checkRows(t, rows, "root@localhost") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") + dbt.MustQuery("CREATE USER user1@localhost") + dbt.MustQuery("GRANT SELECT,INSERT,UPDATE,DELETE ON test.* TO user1@localhost") }) // Test with Network interface connection with all hosts - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.User = "user1" config.DBName = "test" }, - func(dbt *DBTest) { - rows := dbt.mustQuery("select user()") + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") // NOTICE: this is not compatible with MySQL! (MySQL would report user1@localhost also for 127.0.0.1) - cli.checkRows(c, rows, "user1@127.0.0.1") - rows = dbt.mustQuery("show grants") - cli.checkRows(c, rows, "GRANT USAGE ON *.* TO 'user1'@'127.0.0.1'\nGRANT SELECT,INSERT ON test.* TO 'user1'@'127.0.0.1'") + cli.checkRows(t, rows, "user1@127.0.0.1") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'user1'@'127.0.0.1'\nGRANT SELECT,INSERT ON test.* TO 'user1'@'127.0.0.1'") }) // Test with unix domain socket file connection with all hosts - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.Net = "unix" config.Addr = socketFile config.User = "user1" config.DBName = "test" }, - func(dbt *DBTest) { - rows := dbt.mustQuery("select user()") - cli.checkRows(c, rows, "user1@localhost") - rows = dbt.mustQuery("show grants") - cli.checkRows(c, rows, "GRANT USAGE ON *.* TO 'user1'@'localhost'\nGRANT SELECT,INSERT,UPDATE,DELETE ON test.* TO 'user1'@'localhost'") + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") + cli.checkRows(t, rows, "user1@localhost") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'user1'@'localhost'\nGRANT SELECT,INSERT,UPDATE,DELETE ON test.* TO 'user1'@'localhost'") }) } // TestOnlySocket for server configuration without network interface for mysql clients -func (ts *tidbTestSuite) TestOnlySocket(c *C) { +func TestOnlySocket(t *testing.T) { + t.Parallel() osTempDir := os.TempDir() tempDir, err := os.MkdirTemp(osTempDir, "tidb-test.*.socket") - c.Assert(err, IsNil) + require.NoError(t, err) socketFile := tempDir + "/tidbtest.sock" // Unix Socket does not work on Windows, so '/' should be OK defer os.RemoveAll(tempDir) + cli := newTestServerClient() cfg := newTestConfig() cfg.Socket = socketFile cfg.Host = "" // No network interface listening for mysql traffic cfg.Status.ReportStatus = false + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + server, err := NewServer(cfg, ts.tidbdrv) - c.Assert(err, IsNil) + require.NoError(t, err) go func() { err := server.Run() - c.Assert(err, IsNil) + require.NoError(t, err) }() time.Sleep(time.Millisecond * 100) defer server.Close() - c.Assert(server.listener, IsNil) - c.Assert(server.socket, NotNil) + require.Nil(t, server.listener) + require.NotNil(t, server.socket) // Test with Socket connection + Setup user1@% for all host access defer func() { - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.User = "root" config.Net = "unix" config.Addr = socketFile }, - func(dbt *DBTest) { - dbt.mustQuery("DROP USER IF EXISTS 'user1'@'%'") - dbt.mustQuery("DROP USER IF EXISTS 'user1'@'localhost'") - dbt.mustQuery("DROP USER IF EXISTS 'user1'@'127.0.0.1'") + func(dbt *testkit.DBTestKit) { + dbt.MustQuery("DROP USER IF EXISTS 'user1'@'%'") + dbt.MustQuery("DROP USER IF EXISTS 'user1'@'localhost'") + dbt.MustQuery("DROP USER IF EXISTS 'user1'@'127.0.0.1'") }) }() - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.User = "root" config.Net = "unix" config.Addr = socketFile config.DBName = "test" }, - func(dbt *DBTest) { - rows := dbt.mustQuery("select user()") - cli.checkRows(c, rows, "root@localhost") - rows = dbt.mustQuery("show grants") - cli.checkRows(c, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") - dbt.mustQuery("CREATE USER user1@'%'") - dbt.mustQuery("GRANT SELECT ON test.* TO user1@'%'") + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") + cli.checkRows(t, rows, "root@localhost") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") + dbt.MustQuery("CREATE USER user1@'%'") + dbt.MustQuery("GRANT SELECT ON test.* TO user1@'%'") }) // Test with Network interface connection with all hosts, should fail since server not configured db, err := sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { @@ -681,86 +688,86 @@ func (ts *tidbTestSuite) TestOnlySocket(c *C) { config.DBName = "test" config.Addr = "127.0.0.1" })) - c.Assert(err, IsNil, Commentf("Connect succeeded when not configured!?!")) + require.NoErrorf(t, err, "Connect succeeded when not configured!?!") defer db.Close() db, err = sql.Open("mysql", cli.getDSN(func(config *mysql.Config) { config.User = "user1" config.DBName = "test" config.Addr = "127.0.0.1" })) - c.Assert(err, IsNil, Commentf("Connect succeeded when not configured!?!")) + require.NoErrorf(t, err, "Connect succeeded when not configured!?!") defer db.Close() // Test with unix domain socket file connection with all hosts - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.Net = "unix" config.Addr = socketFile config.User = "user1" config.DBName = "test" }, - func(dbt *DBTest) { - rows := dbt.mustQuery("select user()") - cli.checkRows(c, rows, "user1@localhost") - rows = dbt.mustQuery("show grants") - cli.checkRows(c, rows, "GRANT USAGE ON *.* TO 'user1'@'%'\nGRANT SELECT ON test.* TO 'user1'@'%'") + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") + cli.checkRows(t, rows, "user1@localhost") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'user1'@'%'\nGRANT SELECT ON test.* TO 'user1'@'%'") }) // Setup user1@127.0.0.1 for loop back network interface access - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.Net = "unix" config.Addr = socketFile config.User = "root" config.DBName = "test" }, - func(dbt *DBTest) { - rows := dbt.mustQuery("select user()") + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") // NOTICE: this is not compatible with MySQL! (MySQL would report user1@localhost also for 127.0.0.1) - cli.checkRows(c, rows, "root@localhost") - rows = dbt.mustQuery("show grants") - cli.checkRows(c, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") - dbt.mustQuery("CREATE USER user1@127.0.0.1") - dbt.mustQuery("GRANT SELECT,INSERT ON test.* TO user1@'127.0.0.1'") + cli.checkRows(t, rows, "root@localhost") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") + dbt.MustQuery("CREATE USER user1@127.0.0.1") + dbt.MustQuery("GRANT SELECT,INSERT ON test.* TO user1@'127.0.0.1'") }) // Test with unix domain socket file connection with all hosts - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.Net = "unix" config.Addr = socketFile config.User = "user1" config.DBName = "test" }, - func(dbt *DBTest) { - rows := dbt.mustQuery("select user()") - cli.checkRows(c, rows, "user1@localhost") - rows = dbt.mustQuery("show grants") - cli.checkRows(c, rows, "GRANT USAGE ON *.* TO 'user1'@'%'\nGRANT SELECT ON test.* TO 'user1'@'%'") + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") + cli.checkRows(t, rows, "user1@localhost") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'user1'@'%'\nGRANT SELECT ON test.* TO 'user1'@'%'") }) // Setup user1@localhost for socket (and if MySQL compatible; loop back network interface access) - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.Net = "unix" config.Addr = socketFile config.User = "root" config.DBName = "test" }, - func(dbt *DBTest) { - rows := dbt.mustQuery("select user()") - cli.checkRows(c, rows, "root@localhost") - rows = dbt.mustQuery("show grants") - cli.checkRows(c, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") - dbt.mustQuery("CREATE USER user1@localhost") - dbt.mustQuery("GRANT SELECT,INSERT,UPDATE,DELETE ON test.* TO user1@localhost") + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") + cli.checkRows(t, rows, "root@localhost") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION") + dbt.MustQuery("CREATE USER user1@localhost") + dbt.MustQuery("GRANT SELECT,INSERT,UPDATE,DELETE ON test.* TO user1@localhost") }) // Test with unix domain socket file connection with all hosts - cli.runTests(c, func(config *mysql.Config) { + cli.runTests(t, func(config *mysql.Config) { config.Net = "unix" config.Addr = socketFile config.User = "user1" config.DBName = "test" }, - func(dbt *DBTest) { - rows := dbt.mustQuery("select user()") - cli.checkRows(c, rows, "user1@localhost") - rows = dbt.mustQuery("show grants") - cli.checkRows(c, rows, "GRANT USAGE ON *.* TO 'user1'@'localhost'\nGRANT SELECT,INSERT,UPDATE,DELETE ON test.* TO 'user1'@'localhost'") + func(dbt *testkit.DBTestKit) { + rows := dbt.MustQuery("select user()") + cli.checkRows(t, rows, "user1@localhost") + rows = dbt.MustQuery("show grants") + cli.checkRows(t, rows, "GRANT USAGE ON *.* TO 'user1'@'localhost'\nGRANT SELECT,INSERT,UPDATE,DELETE ON test.* TO 'user1'@'localhost'") }) } @@ -868,369 +875,42 @@ func registerTLSConfig(configName string, caCertPath string, clientCertPath stri return mysql.RegisterTLSConfig(configName, tlsConfig) } -func (ts *tidbTestSuite) TestSystemTimeZone(c *C) { - tk := testkit.NewTestKit(c, ts.store) +func TestSystemTimeZone(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + + tk := testkit.NewTestKit(t, ts.store) cfg := newTestConfig() cfg.Socket = "" cfg.Port, cfg.Status.StatusPort = 0, 0 cfg.Status.ReportStatus = false server, err := NewServer(cfg, ts.tidbdrv) - c.Assert(err, IsNil) + require.NoError(t, err) defer server.Close() tz1 := tk.MustQuery("select variable_value from mysql.tidb where variable_name = 'system_tz'").Rows() tk.MustQuery("select @@system_time_zone").Check(tz1) } -func (ts *tidbTestSerialSuite) TestTLSAuto(c *C) { - // Start the server without TLS configure, letting the server create these as AutoTLS is enabled - connOverrider := func(config *mysql.Config) { - config.TLSConfig = "skip-verify" - } - cli := newTestServerClient() - cfg := newTestConfig() - cfg.Socket = "" - cfg.Port = cli.port - cfg.Status.ReportStatus = false - cfg.Security.AutoTLS = true - cfg.Security.RSAKeySize = 528 // Reduces unittest runtime - err := os.MkdirAll(cfg.TempStoragePath, 0700) - c.Assert(err, IsNil) - server, err := NewServer(cfg, ts.tidbdrv) - c.Assert(err, IsNil) - cli.port = getPortFromTCPAddr(server.listener.Addr()) - go func() { - err := server.Run() - c.Assert(err, IsNil) - }() - time.Sleep(time.Millisecond * 100) - err = cli.runTestTLSConnection(c, connOverrider) // Relying on automatically created TLS certificates - c.Assert(err, IsNil) - - server.Close() -} - -func (ts *tidbTestSerialSuite) TestTLSBasic(c *C) { - // Generate valid TLS certificates. - caCert, caKey, err := generateCert(0, "TiDB CA", nil, nil, "/tmp/ca-key.pem", "/tmp/ca-cert.pem") - c.Assert(err, IsNil) - serverCert, _, err := generateCert(1, "tidb-server", caCert, caKey, "/tmp/server-key.pem", "/tmp/server-cert.pem") - c.Assert(err, IsNil) - _, _, err = generateCert(2, "SQL Client Certificate", caCert, caKey, "/tmp/client-key.pem", "/tmp/client-cert.pem") - c.Assert(err, IsNil) - err = registerTLSConfig("client-certificate", "/tmp/ca-cert.pem", "/tmp/client-cert.pem", "/tmp/client-key.pem", "tidb-server", true) - c.Assert(err, IsNil) - - defer func() { - err := os.Remove("/tmp/ca-key.pem") - c.Assert(err, IsNil) - err = os.Remove("/tmp/ca-cert.pem") - c.Assert(err, IsNil) - err = os.Remove("/tmp/server-key.pem") - c.Assert(err, IsNil) - err = os.Remove("/tmp/server-cert.pem") - c.Assert(err, IsNil) - err = os.Remove("/tmp/client-key.pem") - c.Assert(err, IsNil) - err = os.Remove("/tmp/client-cert.pem") - c.Assert(err, IsNil) - }() - - // Start the server with TLS but without CA, in this case the server will not verify client's certificate. - connOverrider := func(config *mysql.Config) { - config.TLSConfig = "skip-verify" - } - cli := newTestServerClient() - cfg := newTestConfig() - cfg.Socket = "" - cfg.Port = cli.port - cfg.Status.ReportStatus = false - cfg.Security = config.Security{ - SSLCert: "/tmp/server-cert.pem", - SSLKey: "/tmp/server-key.pem", - } - server, err := NewServer(cfg, ts.tidbdrv) - c.Assert(err, IsNil) - cli.port = getPortFromTCPAddr(server.listener.Addr()) - go func() { - err := server.Run() - c.Assert(err, IsNil) - }() - time.Sleep(time.Millisecond * 100) - err = cli.runTestTLSConnection(c, connOverrider) // We should establish connection successfully. - c.Assert(err, IsNil) - cli.runTestRegression(c, connOverrider, "TLSRegression") - // Perform server verification. - connOverrider = func(config *mysql.Config) { - config.TLSConfig = "client-certificate" - } - err = cli.runTestTLSConnection(c, connOverrider) // We should establish connection successfully. - c.Assert(err, IsNil, Commentf("%v", errors.ErrorStack(err))) - cli.runTestRegression(c, connOverrider, "TLSRegression") - - // Test SSL/TLS session vars - var v *variable.SessionVars - stats, err := server.Stats(v) - c.Assert(err, IsNil) - c.Assert(stats, HasKey, "Ssl_server_not_after") - c.Assert(stats, HasKey, "Ssl_server_not_before") - c.Assert(stats["Ssl_server_not_after"], Equals, serverCert.NotAfter.Format("Jan _2 15:04:05 2006 MST")) - c.Assert(stats["Ssl_server_not_before"], Equals, serverCert.NotBefore.Format("Jan _2 15:04:05 2006 MST")) - - server.Close() -} - -func (ts *tidbTestSerialSuite) TestTLSVerify(c *C) { - // Generate valid TLS certificates. - caCert, caKey, err := generateCert(0, "TiDB CA", nil, nil, "/tmp/ca-key.pem", "/tmp/ca-cert.pem") - c.Assert(err, IsNil) - _, _, err = generateCert(1, "tidb-server", caCert, caKey, "/tmp/server-key.pem", "/tmp/server-cert.pem") - c.Assert(err, IsNil) - _, _, err = generateCert(2, "SQL Client Certificate", caCert, caKey, "/tmp/client-key.pem", "/tmp/client-cert.pem") - c.Assert(err, IsNil) - err = registerTLSConfig("client-certificate", "/tmp/ca-cert.pem", "/tmp/client-cert.pem", "/tmp/client-key.pem", "tidb-server", true) - c.Assert(err, IsNil) - - defer func() { - err := os.Remove("/tmp/ca-key.pem") - c.Assert(err, IsNil) - err = os.Remove("/tmp/ca-cert.pem") - c.Assert(err, IsNil) - err = os.Remove("/tmp/server-key.pem") - c.Assert(err, IsNil) - err = os.Remove("/tmp/server-cert.pem") - c.Assert(err, IsNil) - err = os.Remove("/tmp/client-key.pem") - c.Assert(err, IsNil) - err = os.Remove("/tmp/client-cert.pem") - c.Assert(err, IsNil) - }() +func TestClientWithCollation(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() - // Start the server with TLS & CA, if the client presents its certificate, the certificate will be verified. - cli := newTestServerClient() - cfg := newTestConfig() - cfg.Socket = "" - cfg.Port = cli.port - cfg.Status.ReportStatus = false - cfg.Security = config.Security{ - SSLCA: "/tmp/ca-cert.pem", - SSLCert: "/tmp/server-cert.pem", - SSLKey: "/tmp/server-key.pem", - } - server, err := NewServer(cfg, ts.tidbdrv) - c.Assert(err, IsNil) - cli.port = getPortFromTCPAddr(server.listener.Addr()) - go func() { - err := server.Run() - c.Assert(err, IsNil) - }() - time.Sleep(time.Millisecond * 100) - // The client does not provide a certificate, the connection should succeed. - err = cli.runTestTLSConnection(c, nil) - c.Assert(err, IsNil) - connOverrider := func(config *mysql.Config) { - config.TLSConfig = "client-certificate" - } - cli.runTestRegression(c, connOverrider, "TLSRegression") - // The client provides a valid certificate. - connOverrider = func(config *mysql.Config) { - config.TLSConfig = "client-certificate" - } - err = cli.runTestTLSConnection(c, connOverrider) - c.Assert(err, IsNil) - cli.runTestRegression(c, connOverrider, "TLSRegression") - server.Close() - - c.Assert(util.IsTLSExpiredError(errors.New("unknown test")), IsFalse) - c.Assert(util.IsTLSExpiredError(x509.CertificateInvalidError{Reason: x509.CANotAuthorizedForThisName}), IsFalse) - c.Assert(util.IsTLSExpiredError(x509.CertificateInvalidError{Reason: x509.Expired}), IsTrue) - - _, _, err = util.LoadTLSCertificates("", "wrong key", "wrong cert", true, 528) - c.Assert(err, NotNil) - _, _, err = util.LoadTLSCertificates("wrong ca", "/tmp/server-key.pem", "/tmp/server-cert.pem", true, 528) - c.Assert(err, NotNil) + ts.runTestClientWithCollation(t) } -func (ts *tidbTestSerialSuite) TestReloadTLS(c *C) { - // Generate valid TLS certificates. - caCert, caKey, err := generateCert(0, "TiDB CA", nil, nil, "/tmp/ca-key-reload.pem", "/tmp/ca-cert-reload.pem") - c.Assert(err, IsNil) - _, _, err = generateCert(1, "tidb-server", caCert, caKey, "/tmp/server-key-reload.pem", "/tmp/server-cert-reload.pem") - c.Assert(err, IsNil) - _, _, err = generateCert(2, "SQL Client Certificate", caCert, caKey, "/tmp/client-key-reload.pem", "/tmp/client-cert-reload.pem") - c.Assert(err, IsNil) - err = registerTLSConfig("client-certificate-reload", "/tmp/ca-cert-reload.pem", "/tmp/client-cert-reload.pem", "/tmp/client-key-reload.pem", "tidb-server", true) - c.Assert(err, IsNil) - - defer func() { - os.Remove("/tmp/ca-key-reload.pem") - os.Remove("/tmp/ca-cert-reload.pem") - - os.Remove("/tmp/server-key-reload.pem") - os.Remove("/tmp/server-cert-reload.pem") - os.Remove("/tmp/client-key-reload.pem") - os.Remove("/tmp/client-cert-reload.pem") - }() +func TestCreateTableFlen(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() - // try old cert used in startup configuration. - cli := newTestServerClient() - cfg := newTestConfig() - cfg.Socket = "" - cfg.Port = cli.port - cfg.Status.ReportStatus = false - cfg.Security = config.Security{ - SSLCA: "/tmp/ca-cert-reload.pem", - SSLCert: "/tmp/server-cert-reload.pem", - SSLKey: "/tmp/server-key-reload.pem", - } - server, err := NewServer(cfg, ts.tidbdrv) - c.Assert(err, IsNil) - cli.port = getPortFromTCPAddr(server.listener.Addr()) - go func() { - err := server.Run() - c.Assert(err, IsNil) - }() - time.Sleep(time.Millisecond * 100) - // The client provides a valid certificate. - connOverrider := func(config *mysql.Config) { - config.TLSConfig = "client-certificate-reload" - } - err = cli.runTestTLSConnection(c, connOverrider) - c.Assert(err, IsNil) - - // try reload a valid cert. - tlsCfg := server.getTLSConfig() - cert, err := x509.ParseCertificate(tlsCfg.Certificates[0].Certificate[0]) - c.Assert(err, IsNil) - oldExpireTime := cert.NotAfter - _, _, err = generateCert(1, "tidb-server", caCert, caKey, "/tmp/server-key-reload2.pem", "/tmp/server-cert-reload2.pem", func(c *x509.Certificate) { - c.NotBefore = time.Now().Add(-24 * time.Hour).UTC() - c.NotAfter = time.Now().Add(1 * time.Hour).UTC() - }) - c.Assert(err, IsNil) - err = os.Rename("/tmp/server-key-reload2.pem", "/tmp/server-key-reload.pem") - c.Assert(err, IsNil) - err = os.Rename("/tmp/server-cert-reload2.pem", "/tmp/server-cert-reload.pem") - c.Assert(err, IsNil) - connOverrider = func(config *mysql.Config) { - config.TLSConfig = "skip-verify" - } - err = cli.runReloadTLS(c, connOverrider, false) - c.Assert(err, IsNil) - connOverrider = func(config *mysql.Config) { - config.TLSConfig = "client-certificate-reload" - } - err = cli.runTestTLSConnection(c, connOverrider) - c.Assert(err, IsNil) - - tlsCfg = server.getTLSConfig() - cert, err = x509.ParseCertificate(tlsCfg.Certificates[0].Certificate[0]) - c.Assert(err, IsNil) - newExpireTime := cert.NotAfter - c.Assert(newExpireTime.After(oldExpireTime), IsTrue) - - // try reload a expired cert. - _, _, err = generateCert(1, "tidb-server", caCert, caKey, "/tmp/server-key-reload3.pem", "/tmp/server-cert-reload3.pem", func(c *x509.Certificate) { - c.NotBefore = time.Now().Add(-24 * time.Hour).UTC() - c.NotAfter = c.NotBefore.Add(1 * time.Hour).UTC() - }) - c.Assert(err, IsNil) - err = os.Rename("/tmp/server-key-reload3.pem", "/tmp/server-key-reload.pem") - c.Assert(err, IsNil) - err = os.Rename("/tmp/server-cert-reload3.pem", "/tmp/server-cert-reload.pem") - c.Assert(err, IsNil) - connOverrider = func(config *mysql.Config) { - config.TLSConfig = "skip-verify" - } - err = cli.runReloadTLS(c, connOverrider, false) - c.Assert(err, IsNil) - connOverrider = func(config *mysql.Config) { - config.TLSConfig = "client-certificate-reload" - } - err = cli.runTestTLSConnection(c, connOverrider) - c.Assert(err, NotNil) - c.Assert(util.IsTLSExpiredError(err), IsTrue, Commentf("real error is %+v", err)) - server.Close() -} - -func (ts *tidbTestSerialSuite) TestErrorNoRollback(c *C) { - // Generate valid TLS certificates. - caCert, caKey, err := generateCert(0, "TiDB CA", nil, nil, "/tmp/ca-key-rollback.pem", "/tmp/ca-cert-rollback.pem") - c.Assert(err, IsNil) - _, _, err = generateCert(1, "tidb-server", caCert, caKey, "/tmp/server-key-rollback.pem", "/tmp/server-cert-rollback.pem") - c.Assert(err, IsNil) - _, _, err = generateCert(2, "SQL Client Certificate", caCert, caKey, "/tmp/client-key-rollback.pem", "/tmp/client-cert-rollback.pem") - c.Assert(err, IsNil) - err = registerTLSConfig("client-cert-rollback-test", "/tmp/ca-cert-rollback.pem", "/tmp/client-cert-rollback.pem", "/tmp/client-key-rollback.pem", "tidb-server", true) - c.Assert(err, IsNil) - - defer func() { - os.Remove("/tmp/ca-key-rollback.pem") - os.Remove("/tmp/ca-cert-rollback.pem") - - os.Remove("/tmp/server-key-rollback.pem") - os.Remove("/tmp/server-cert-rollback.pem") - os.Remove("/tmp/client-key-rollback.pem") - os.Remove("/tmp/client-cert-rollback.pem") - }() - - cli := newTestServerClient() - cfg := newTestConfig() - cfg.Socket = "" - cfg.Port = cli.port - cfg.Status.ReportStatus = false - - cfg.Security = config.Security{ - RequireSecureTransport: true, - SSLCA: "wrong path", - SSLCert: "wrong path", - SSLKey: "wrong path", - } - _, err = NewServer(cfg, ts.tidbdrv) - c.Assert(err, NotNil) - - // test reload tls fail with/without "error no rollback option" - cfg.Security = config.Security{ - SSLCA: "/tmp/ca-cert-rollback.pem", - SSLCert: "/tmp/server-cert-rollback.pem", - SSLKey: "/tmp/server-key-rollback.pem", - } - server, err := NewServer(cfg, ts.tidbdrv) - c.Assert(err, IsNil) - cli.port = getPortFromTCPAddr(server.listener.Addr()) - go func() { - err := server.Run() - c.Assert(err, IsNil) - }() - defer server.Close() - time.Sleep(time.Millisecond * 100) - connOverrider := func(config *mysql.Config) { - config.TLSConfig = "client-cert-rollback-test" - } - err = cli.runTestTLSConnection(c, connOverrider) - c.Assert(err, IsNil) - os.Remove("/tmp/server-key-rollback.pem") - err = cli.runReloadTLS(c, connOverrider, false) - c.Assert(err, NotNil) - tlsCfg := server.getTLSConfig() - c.Assert(tlsCfg, NotNil) - err = cli.runReloadTLS(c, connOverrider, true) - c.Assert(err, IsNil) - tlsCfg = server.getTLSConfig() - c.Assert(tlsCfg, IsNil) -} - -func (ts *tidbTestSuite) TestClientWithCollation(c *C) { - c.Parallel() - ts.runTestClientWithCollation(c) -} - -func (ts *tidbTestSuite) TestCreateTableFlen(c *C) { // issue #4540 qctx, err := ts.tidbdrv.OpenCtx(uint64(0), 0, uint8(tmysql.DefaultCollationID), "test", nil) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = Execute(context.Background(), qctx, "use test;") - c.Assert(err, IsNil) + require.NoError(t, err) ctx := context.Background() testSQL := "CREATE TABLE `t1` (" + @@ -1263,25 +943,25 @@ func (ts *tidbTestSuite) TestCreateTableFlen(c *C) { "PRIMARY KEY (`a`)" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin" _, err = Execute(ctx, qctx, testSQL) - c.Assert(err, IsNil) + require.NoError(t, err) rs, err := Execute(ctx, qctx, "show create table t1") - c.Assert(err, IsNil) + require.NoError(t, err) req := rs.NewChunk(nil) err = rs.Next(ctx, req) - c.Assert(err, IsNil) + require.NoError(t, err) cols := rs.Columns() - c.Assert(err, IsNil) - c.Assert(len(cols), Equals, 2) - c.Assert(int(cols[0].ColumnLength), Equals, 5*tmysql.MaxBytesOfCharacter) - c.Assert(int(cols[1].ColumnLength), Equals, len(req.GetRow(0).GetString(1))*tmysql.MaxBytesOfCharacter) + require.NoError(t, err) + require.Len(t, cols, 2) + require.Equal(t, 5*tmysql.MaxBytesOfCharacter, int(cols[0].ColumnLength)) + require.Equal(t, len(req.GetRow(0).GetString(1))*tmysql.MaxBytesOfCharacter, int(cols[1].ColumnLength)) // for issue#5246 rs, err = Execute(ctx, qctx, "select y, z from t1") - c.Assert(err, IsNil) + require.NoError(t, err) cols = rs.Columns() - c.Assert(len(cols), Equals, 2) - c.Assert(int(cols[0].ColumnLength), Equals, 21) - c.Assert(int(cols[1].ColumnLength), Equals, 22) + require.Len(t, cols, 2) + require.Equal(t, 21, int(cols[0].ColumnLength)) + require.Equal(t, 22, int(cols[1].ColumnLength)) } func Execute(ctx context.Context, qc *TiDBContext, sql string) (ResultSet, error) { @@ -1295,39 +975,47 @@ func Execute(ctx context.Context, qc *TiDBContext, sql string) (ResultSet, error return qc.ExecuteStmt(ctx, stmts[0]) } -func (ts *tidbTestSuite) TestShowTablesFlen(c *C) { +func TestShowTablesFlen(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + qctx, err := ts.tidbdrv.OpenCtx(uint64(0), 0, uint8(tmysql.DefaultCollationID), "test", nil) - c.Assert(err, IsNil) + require.NoError(t, err) ctx := context.Background() _, err = Execute(ctx, qctx, "use test;") - c.Assert(err, IsNil) + require.NoError(t, err) testSQL := "create table abcdefghijklmnopqrstuvwxyz (i int)" _, err = Execute(ctx, qctx, testSQL) - c.Assert(err, IsNil) + require.NoError(t, err) rs, err := Execute(ctx, qctx, "show tables") - c.Assert(err, IsNil) + require.NoError(t, err) req := rs.NewChunk(nil) err = rs.Next(ctx, req) - c.Assert(err, IsNil) + require.NoError(t, err) cols := rs.Columns() - c.Assert(err, IsNil) - c.Assert(len(cols), Equals, 1) - c.Assert(int(cols[0].ColumnLength), Equals, 26*tmysql.MaxBytesOfCharacter) + require.NoError(t, err) + require.Len(t, cols, 1) + require.Equal(t, 26*tmysql.MaxBytesOfCharacter, int(cols[0].ColumnLength)) } -func checkColNames(c *C, columns []*ColumnInfo, names ...string) { +func checkColNames(t *testing.T, columns []*ColumnInfo, names ...string) { for i, name := range names { - c.Assert(columns[i].Name, Equals, name) - c.Assert(columns[i].OrgName, Equals, name) + require.Equal(t, name, columns[i].Name) + require.Equal(t, name, columns[i].OrgName) } } -func (ts *tidbTestSuite) TestFieldList(c *C) { +func TestFieldList(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + qctx, err := ts.tidbdrv.OpenCtx(uint64(0), 0, uint8(tmysql.DefaultCollationID), "test", nil) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = Execute(context.Background(), qctx, "use test;") - c.Assert(err, IsNil) + require.NoError(t, err) ctx := context.Background() testSQL := `create table t ( @@ -1352,22 +1040,22 @@ func (ts *tidbTestSuite) TestFieldList(c *C) { c_year year )` _, err = Execute(ctx, qctx, testSQL) - c.Assert(err, IsNil) + require.NoError(t, err) colInfos, err := qctx.FieldList("t") - c.Assert(err, IsNil) - c.Assert(len(colInfos), Equals, 19) + require.NoError(t, err) + require.Len(t, colInfos, 19) - checkColNames(c, colInfos, "c_bit", "c_int_d", "c_bigint_d", "c_float_d", + checkColNames(t, colInfos, "c_bit", "c_int_d", "c_bigint_d", "c_float_d", "c_double_d", "c_decimal", "c_datetime", "c_time", "c_date", "c_timestamp", "c_char", "c_varchar", "c_text_d", "c_binary", "c_blob_d", "c_set", "c_enum", "c_json", "c_year") for _, cols := range colInfos { - c.Assert(cols.Schema, Equals, "test") + require.Equal(t, "test", cols.Schema) } for _, cols := range colInfos { - c.Assert(cols.Table, Equals, "t") + require.Equal(t, "t", cols.Table) } for i, col := range colInfos { @@ -1375,140 +1063,152 @@ func (ts *tidbTestSuite) TestFieldList(c *C) { case 10, 11, 12, 15, 16: // c_char char(20), c_varchar varchar(20), c_text_d text, // c_set set('a', 'b', 'c'), c_enum enum('a', 'b', 'c') - c.Assert(col.Charset, Equals, uint16(tmysql.CharsetNameToID(tmysql.DefaultCharset)), Commentf("index %d", i)) + require.Equalf(t, uint16(tmysql.CharsetNameToID(tmysql.DefaultCharset)), col.Charset, "index %d", i) continue } - c.Assert(col.Charset, Equals, uint16(tmysql.CharsetNameToID("binary")), Commentf("index %d", i)) + require.Equalf(t, uint16(tmysql.CharsetNameToID("binary")), col.Charset, "index %d", i) } // c_decimal decimal(6, 3) - c.Assert(colInfos[5].Decimal, Equals, uint8(3)) + require.Equal(t, uint8(3), colInfos[5].Decimal) // for issue#10513 tooLongColumnAsName := "COALESCE(0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0)" columnAsName := tooLongColumnAsName[:tmysql.MaxAliasIdentifierLen] rs, err := Execute(ctx, qctx, "select "+tooLongColumnAsName) - c.Assert(err, IsNil) + require.NoError(t, err) cols := rs.Columns() - c.Assert(cols[0].OrgName, Equals, tooLongColumnAsName) - c.Assert(cols[0].Name, Equals, columnAsName) + require.Equal(t, tooLongColumnAsName, cols[0].OrgName) + require.Equal(t, columnAsName, cols[0].Name) rs, err = Execute(ctx, qctx, "select c_bit as '"+tooLongColumnAsName+"' from t") - c.Assert(err, IsNil) + require.NoError(t, err) cols = rs.Columns() - c.Assert(cols[0].OrgName, Equals, "c_bit") - c.Assert(cols[0].Name, Equals, columnAsName) + require.Equal(t, "c_bit", cols[0].OrgName) + require.Equal(t, columnAsName, cols[0].Name) } -func (ts *tidbTestSuite) TestClientErrors(c *C) { - ts.runTestInfoschemaClientErrors(c) +func TestClientErrors(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + ts.runTestInfoschemaClientErrors(t) } -func (ts *tidbTestSuite) TestInitConnect(c *C) { - ts.runTestInitConnect(c) +func TestInitConnect(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + ts.runTestInitConnect(t) } -func (ts *tidbTestSuite) TestSumAvg(c *C) { - c.Parallel() - ts.runTestSumAvg(c) +func TestSumAvg(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + ts.runTestSumAvg(t) } -func (ts *tidbTestSuite) TestNullFlag(c *C) { +func TestNullFlag(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + qctx, err := ts.tidbdrv.OpenCtx(uint64(0), 0, uint8(tmysql.DefaultCollationID), "test", nil) - c.Assert(err, IsNil) + require.NoError(t, err) ctx := context.Background() { // issue #9689 rs, err := Execute(ctx, qctx, "select 1") - c.Assert(err, IsNil) + require.NoError(t, err) cols := rs.Columns() - c.Assert(len(cols), Equals, 1) + require.Len(t, cols, 1) expectFlag := uint16(tmysql.NotNullFlag | tmysql.BinaryFlag) - c.Assert(dumpFlag(cols[0].Type, cols[0].Flag), Equals, expectFlag) + require.Equal(t, expectFlag, dumpFlag(cols[0].Type, cols[0].Flag)) } { // issue #19025 rs, err := Execute(ctx, qctx, "select convert('{}', JSON)") - c.Assert(err, IsNil) + require.NoError(t, err) cols := rs.Columns() - c.Assert(len(cols), Equals, 1) + require.Len(t, cols, 1) expectFlag := uint16(tmysql.BinaryFlag) - c.Assert(dumpFlag(cols[0].Type, cols[0].Flag), Equals, expectFlag) + require.Equal(t, expectFlag, dumpFlag(cols[0].Type, cols[0].Flag)) } { // issue #18488 _, err := Execute(ctx, qctx, "use test") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = Execute(ctx, qctx, "CREATE TABLE `test` (`iD` bigint(20) NOT NULL, `INT_TEST` int(11) DEFAULT NULL);") - c.Assert(err, IsNil) + require.NoError(t, err) rs, err := Execute(ctx, qctx, `SELECT id + int_test as res FROM test GROUP BY res ORDER BY res;`) - c.Assert(err, IsNil) + require.NoError(t, err) cols := rs.Columns() - c.Assert(len(cols), Equals, 1) + require.Len(t, cols, 1) expectFlag := uint16(tmysql.BinaryFlag) - c.Assert(dumpFlag(cols[0].Type, cols[0].Flag), Equals, expectFlag) + require.Equal(t, expectFlag, dumpFlag(cols[0].Type, cols[0].Flag)) } { rs, err := Execute(ctx, qctx, "select if(1, null, 1) ;") - c.Assert(err, IsNil) + require.NoError(t, err) cols := rs.Columns() - c.Assert(len(cols), Equals, 1) + require.Len(t, cols, 1) expectFlag := uint16(tmysql.BinaryFlag) - c.Assert(dumpFlag(cols[0].Type, cols[0].Flag), Equals, expectFlag) + require.Equal(t, expectFlag, dumpFlag(cols[0].Type, cols[0].Flag)) } { rs, err := Execute(ctx, qctx, "select CASE 1 WHEN 2 THEN 1 END ;") - c.Assert(err, IsNil) + require.NoError(t, err) cols := rs.Columns() - c.Assert(len(cols), Equals, 1) + require.Len(t, cols, 1) expectFlag := uint16(tmysql.BinaryFlag) - c.Assert(dumpFlag(cols[0].Type, cols[0].Flag), Equals, expectFlag) + require.Equal(t, expectFlag, dumpFlag(cols[0].Type, cols[0].Flag)) } { rs, err := Execute(ctx, qctx, "select NULL;") - c.Assert(err, IsNil) + require.NoError(t, err) cols := rs.Columns() - c.Assert(len(cols), Equals, 1) + require.Len(t, cols, 1) expectFlag := uint16(tmysql.BinaryFlag) - c.Assert(dumpFlag(cols[0].Type, cols[0].Flag), Equals, expectFlag) + require.Equal(t, expectFlag, dumpFlag(cols[0].Type, cols[0].Flag)) } } -func (ts *tidbTestSuite) TestNO_DEFAULT_VALUEFlag(c *C) { +func TestNO_DEFAULT_VALUEFlag(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + // issue #21465 qctx, err := ts.tidbdrv.OpenCtx(uint64(0), 0, uint8(tmysql.DefaultCollationID), "test", nil) - c.Assert(err, IsNil) + require.NoError(t, err) ctx := context.Background() _, err = Execute(ctx, qctx, "use test") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = Execute(ctx, qctx, "drop table if exists t") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = Execute(ctx, qctx, "create table t(c1 int key, c2 int);") - c.Assert(err, IsNil) + require.NoError(t, err) rs, err := Execute(ctx, qctx, "select c1 from t;") - c.Assert(err, IsNil) + require.NoError(t, err) cols := rs.Columns() - c.Assert(len(cols), Equals, 1) + require.Len(t, cols, 1) expectFlag := uint16(tmysql.NotNullFlag | tmysql.PriKeyFlag | tmysql.NoDefaultValueFlag) - c.Assert(dumpFlag(cols[0].Type, cols[0].Flag), Equals, expectFlag) + require.Equal(t, expectFlag, dumpFlag(cols[0].Type, cols[0].Flag)) } -func (ts *tidbTestSuite) TestGracefulShutdown(c *C) { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - defer store.Close() - session.DisableStats4Test() - dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - defer dom.Close() - ts.tidbdrv = NewTiDBDriver(ts.store) +func TestGracefulShutdown(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() + cli := newTestServerClient() cfg := newTestConfig() cfg.Socket = "" @@ -1518,136 +1218,100 @@ func (ts *tidbTestSuite) TestGracefulShutdown(c *C) { cfg.Status.ReportStatus = true cfg.Performance.TCPKeepAlive = true server, err := NewServer(cfg, ts.tidbdrv) - c.Assert(err, IsNil) - c.Assert(server, NotNil) + require.NoError(t, err) + require.NotNil(t, server) cli.port = getPortFromTCPAddr(server.listener.Addr()) cli.statusPort = getPortFromTCPAddr(server.statusListener.Addr()) go func() { err := server.Run() - c.Assert(err, IsNil) + require.NoError(t, err) }() time.Sleep(time.Millisecond * 100) resp, err := cli.fetchStatus("/status") // server is up - c.Assert(err, IsNil) - c.Assert(resp.Body.Close(), IsNil) + require.NoError(t, err) + require.Nil(t, resp.Body.Close()) go server.Close() time.Sleep(time.Millisecond * 500) resp, _ = cli.fetchStatus("/status") // should return 5xx code - c.Assert(resp.StatusCode, Equals, 500) - c.Assert(resp.Body.Close(), IsNil) + require.Equal(t, 500, resp.StatusCode) + require.Nil(t, resp.Body.Close()) time.Sleep(time.Second * 2) // nolint: bodyclose _, err = cli.fetchStatus("/status") // status is gone - c.Assert(err, ErrorMatches, ".*connect: connection refused") + require.Regexp(t, ".*connect: connection refused", err.Error()) } -func (ts *tidbTestSerialSuite) TestDefaultCharacterAndCollation(c *C) { - // issue #21194 - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - // 255 is the collation id of mysql client 8 default collation_connection - qctx, err := ts.tidbdrv.OpenCtx(uint64(0), 0, uint8(255), "test", nil) - c.Assert(err, IsNil) - testCase := []struct { - variable string - except string - }{ - {"collation_connection", "utf8mb4_bin"}, - {"character_set_connection", "utf8mb4"}, - {"character_set_client", "utf8mb4"}, - } +func TestPessimisticInsertSelectForUpdate(t *testing.T) { + t.Parallel() + ts, cleanup := createTidbTestSuite(t) + defer cleanup() - for _, t := range testCase { - sVars, b := qctx.GetSessionVars().GetSystemVar(t.variable) - c.Assert(b, IsTrue) - c.Assert(sVars, Equals, t.except) - } -} - -func (ts *tidbTestSuite) TestPessimisticInsertSelectForUpdate(c *C) { qctx, err := ts.tidbdrv.OpenCtx(uint64(0), 0, uint8(tmysql.DefaultCollationID), "test", nil) - c.Assert(err, IsNil) + require.NoError(t, err) defer qctx.Close() ctx := context.Background() _, err = Execute(ctx, qctx, "use test;") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = Execute(ctx, qctx, "drop table if exists t1, t2") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = Execute(ctx, qctx, "create table t1 (id int)") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = Execute(ctx, qctx, "create table t2 (id int)") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = Execute(ctx, qctx, "insert into t1 select 1") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = Execute(ctx, qctx, "begin pessimistic") - c.Assert(err, IsNil) + require.NoError(t, err) rs, err := Execute(ctx, qctx, "INSERT INTO t2 (id) select id from t1 where id = 1 for update") - c.Assert(err, IsNil) - c.Assert(rs, IsNil) // should be no delay -} - -func (ts *tidbTestSerialSuite) TestPrepareCount(c *C) { - qctx, err := ts.tidbdrv.OpenCtx(uint64(0), 0, uint8(tmysql.DefaultCollationID), "test", nil) - c.Assert(err, IsNil) - prepareCnt := atomic.LoadInt64(&variable.PreparedStmtCount) - ctx := context.Background() - _, err = Execute(ctx, qctx, "use test;") - c.Assert(err, IsNil) - _, err = Execute(ctx, qctx, "drop table if exists t1") - c.Assert(err, IsNil) - _, err = Execute(ctx, qctx, "create table t1 (id int)") - c.Assert(err, IsNil) - stmt, _, _, err := qctx.Prepare("insert into t1 values (?)") - c.Assert(err, IsNil) - c.Assert(atomic.LoadInt64(&variable.PreparedStmtCount), Equals, prepareCnt+1) - c.Assert(err, IsNil) - err = qctx.GetStatement(stmt.ID()).Close() - c.Assert(err, IsNil) - c.Assert(atomic.LoadInt64(&variable.PreparedStmtCount), Equals, prepareCnt) + require.NoError(t, err) + require.Nil(t, rs) // should be no delay } type collectorWrapper struct { reporter.TopSQLReporter } -func (ts *tidbTestTopSQLSuite) TestTopSQLCPUProfile(c *C) { +func TestTopSQLCPUProfile(t *testing.T) { + ts, cleanup := createTidbTestTopSQLSuite(t) + defer cleanup() + db, err := sql.Open("mysql", ts.getDSN()) - c.Assert(err, IsNil, Commentf("Error connecting")) + require.NoErrorf(t, err, "Error connecting") defer func() { err := db.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() - c.Assert(failpoint.Enable("github.com/pingcap/tidb/domain/skipLoadSysVarCacheLoop", `return(true)`), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/util/topsql/mockHighLoadForEachSQL", `return(true)`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/domain/skipLoadSysVarCacheLoop", `return(true)`)) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/util/topsql/mockHighLoadForEachSQL", `return(true)`)) defer func() { err = failpoint.Disable("github.com/pingcap/tidb/domain/skipLoadSysVarCacheLoop") - c.Assert(err, IsNil) + require.NoError(t, err) err = failpoint.Disable("github.com/pingcap/tidb/util/topsql/mockHighLoadForEachSQL") - c.Assert(err, IsNil) + require.NoError(t, err) }() collector := mockTopSQLTraceCPU.NewTopSQLCollector() tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{collector}) - dbt := &DBTest{c, db} - dbt.mustExec("drop database if exists topsql") - dbt.mustExec("create database topsql") - dbt.mustExec("use topsql;") - dbt.mustExec("create table t (a int auto_increment, b int, unique index idx(a));") - dbt.mustExec("create table t1 (a int auto_increment, b int, unique index idx(a));") - dbt.mustExec("create table t2 (a int auto_increment, b int, unique index idx(a));") - dbt.mustExec("set @@global.tidb_enable_top_sql='On';") + dbt := testkit.NewDBTestKit(t, db) + dbt.MustExec("drop database if exists topsql") + dbt.MustExec("create database topsql") + dbt.MustExec("use topsql;") + dbt.MustExec("create table t (a int auto_increment, b int, unique index idx(a));") + dbt.MustExec("create table t1 (a int auto_increment, b int, unique index idx(a));") + dbt.MustExec("create table t2 (a int auto_increment, b int, unique index idx(a));") + dbt.MustExec("set @@global.tidb_enable_top_sql='On';") config.UpdateGlobal(func(conf *config.Config) { conf.TopSQL.ReceiverAddress = "127.0.0.1:4001" }) - dbt.mustExec("set @@global.tidb_top_sql_precision_seconds=1;") - dbt.mustExec("set @@global.tidb_txn_mode = 'pessimistic'") + dbt.MustExec("set @@global.tidb_top_sql_precision_seconds=1;") + dbt.MustExec("set @@global.tidb_txn_mode = 'pessimistic'") // Test case 1: DML query: insert/update/replace/delete/select cases1 := []struct { @@ -1671,10 +1335,10 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLCPUProfile(c *C) { ctx, cancel := context.WithCancel(context.Background()) cases1[i].cancel = cancel sqlStr := ca.sql - go ts.loopExec(ctx, c, func(db *sql.DB) { - dbt := &DBTest{c, db} + go ts.loopExec(ctx, t, func(db *sql.DB) { + dbt := testkit.NewDBTestKit(t, db) if strings.HasPrefix(sqlStr, "select") { - rows := dbt.mustQuery(sqlStr) + rows := dbt.MustQuery(sqlStr) for rows.Next() { } } else { @@ -1687,25 +1351,24 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLCPUProfile(c *C) { timeoutCtx, cancel := context.WithTimeout(context.Background(), time.Second*20) defer cancel() checkFn := func(sql, planRegexp string) { - c.Assert(timeoutCtx.Err(), IsNil) - commentf := Commentf("sql: %v", sql) + require.NoError(t, timeoutCtx.Err()) stats := collector.GetSQLStatsBySQLWithRetry(sql, len(planRegexp) > 0) // since 1 sql may has many plan, check `len(stats) > 0` instead of `len(stats) == 1`. - c.Assert(len(stats) > 0, IsTrue, commentf) + require.Greaterf(t, len(stats), 0, "sql: %v", sql) for _, s := range stats { sqlStr := collector.GetSQL(s.SQLDigest) encodedPlan := collector.GetPlan(s.PlanDigest) // Normalize the user SQL before check. normalizedSQL := parser.Normalize(sql) - c.Assert(sqlStr, Equals, normalizedSQL, commentf) + require.Equalf(t, normalizedSQL, sqlStr, "sql: %v", sql) // decode plan before check. normalizedPlan, err := plancodec.DecodeNormalizedPlan(encodedPlan) - c.Assert(err, IsNil) + require.NoError(t, err) // remove '\n' '\t' before do regexp match. normalizedPlan = strings.Replace(normalizedPlan, "\n", " ", -1) normalizedPlan = strings.Replace(normalizedPlan, "\t", " ", -1) - c.Assert(normalizedPlan, Matches, planRegexp, commentf) + require.Regexpf(t, planRegexp, normalizedPlan, "sql: %v", sql) } } // Wait the top sql collector to collect profile data. @@ -1740,14 +1403,14 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLCPUProfile(c *C) { cases2[i].cancel = cancel prepare, args := ca.prepare, ca.args var stmt *sql.Stmt - go ts.loopExec(ctx, c, func(db *sql.DB) { + go ts.loopExec(ctx, t, func(db *sql.DB) { if stmt == nil { stmt, err = db.Prepare(prepare) - c.Assert(err, IsNil) + require.NoError(t, err) } if strings.HasPrefix(prepare, "select") { rows, err := stmt.Query(args...) - c.Assert(err, IsNil) + require.NoError(t, err) for rows.Next() { } } else { @@ -1787,17 +1450,17 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLCPUProfile(c *C) { cases3[i].cancel = cancel prepare, args := ca.prepare, ca.args doPrepare := true - go ts.loopExec(ctx, c, func(db *sql.DB) { + go ts.loopExec(ctx, t, func(db *sql.DB) { if doPrepare { doPrepare = false _, err := db.Exec(fmt.Sprintf("prepare stmt from '%v'", prepare)) - c.Assert(err, IsNil) + require.NoError(t, err) } sqlBuf := bytes.NewBuffer(nil) sqlBuf.WriteString("execute stmt ") for i := range args { _, err = db.Exec(fmt.Sprintf("set @%c=%v", 'a'+i, args[i])) - c.Assert(err, IsNil) + require.NoError(t, err) if i == 0 { sqlBuf.WriteString("using ") } else { @@ -1808,7 +1471,7 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLCPUProfile(c *C) { } if strings.HasPrefix(prepare, "select") { rows, err := db.Query(sqlBuf.String()) - c.Assert(err, IsNil, Commentf("%v", sqlBuf.String())) + require.NoErrorf(t, err, "%v", sqlBuf.String()) for rows.Next() { } } else { @@ -1829,7 +1492,7 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLCPUProfile(c *C) { // Test case 4: transaction commit ctx4, cancel4 := context.WithCancel(context.Background()) defer cancel4() - go ts.loopExec(ctx4, c, func(db *sql.DB) { + go ts.loopExec(ctx4, t, func(db *sql.DB) { db.Exec("begin") db.Exec("insert into t () values (),(),(),(),(),(),(),(),(),(),(),(),(),(),(),(),(),(),(),(),(),()") db.Exec("commit") @@ -1838,40 +1501,43 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLCPUProfile(c *C) { checkFn("commit", "") } -func (ts *tidbTestTopSQLSuite) TestTopSQLAgent(c *C) { - c.Skip("unstable, skip it and fix it before 20210702") +func TestTopSQLAgent(t *testing.T) { + t.Skip("unstable, skip it and fix it before 20210702") + + ts, cleanup := createTidbTestTopSQLSuite(t) + defer cleanup() db, err := sql.Open("mysql", ts.getDSN()) - c.Assert(err, IsNil, Commentf("Error connecting")) + require.NoError(t, err, "Error connecting") defer func() { err := db.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() agentServer, err := mockTopSQLReporter.StartMockAgentServer() - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { agentServer.Stop() }() - c.Assert(failpoint.Enable("github.com/pingcap/tidb/util/topsql/reporter/resetTimeoutForTest", `return(true)`), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/domain/skipLoadSysVarCacheLoop", `return(true)`), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/util/topsql/mockHighLoadForEachSQL", `return(true)`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/util/topsql/reporter/resetTimeoutForTest", `return(true)`)) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/domain/skipLoadSysVarCacheLoop", `return(true)`)) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/util/topsql/mockHighLoadForEachSQL", `return(true)`)) defer func() { err := failpoint.Disable("github.com/pingcap/tidb/util/topsql/reporter/resetTimeoutForTest") - c.Assert(err, IsNil) + require.NoError(t, err) err = failpoint.Disable("github.com/pingcap/tidb/domain/skipLoadSysVarCacheLoop") - c.Assert(err, IsNil) + require.NoError(t, err) err = failpoint.Disable("github.com/pingcap/tidb/util/topsql/mockHighLoadForEachSQL") - c.Assert(err, IsNil) + require.NoError(t, err) }() - dbt := &DBTest{c, db} - dbt.mustExec("drop database if exists topsql") - dbt.mustExec("create database topsql") - dbt.mustExec("use topsql;") + dbt := testkit.NewDBTestKit(t, db) + dbt.MustExec("drop database if exists topsql") + dbt.MustExec("create database topsql") + dbt.MustExec("use topsql;") for i := 0; i < 20; i++ { - dbt.mustExec(fmt.Sprintf("create table t%v (a int auto_increment, b int, unique index idx(a));", i)) + dbt.MustExec(fmt.Sprintf("create table t%v (a int auto_increment, b int, unique index idx(a));", i)) for j := 0; j < 100; j++ { - dbt.mustExec(fmt.Sprintf("insert into t%v (b) values (%v);", i, j)) + dbt.MustExec(fmt.Sprintf("insert into t%v (b) values (%v);", i, j)) } } setTopSQLReceiverAddress := func(addr string) { @@ -1879,11 +1545,11 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLAgent(c *C) { conf.TopSQL.ReceiverAddress = addr }) } - dbt.mustExec("set @@global.tidb_enable_top_sql='On';") + dbt.MustExec("set @@global.tidb_enable_top_sql='On';") setTopSQLReceiverAddress("") - dbt.mustExec("set @@global.tidb_top_sql_precision_seconds=1;") - dbt.mustExec("set @@global.tidb_top_sql_report_interval_seconds=2;") - dbt.mustExec("set @@global.tidb_top_sql_max_statement_count=5;") + dbt.MustExec("set @@global.tidb_top_sql_precision_seconds=1;") + dbt.MustExec("set @@global.tidb_top_sql_report_interval_seconds=2;") + dbt.MustExec("set @@global.tidb_top_sql_max_statement_count=5;") r := reporter.NewRemoteTopSQLReporter(reporter.NewGRPCReportClient(plancodec.DecodeNormalizedPlan)) tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{r}) @@ -1891,28 +1557,28 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLAgent(c *C) { // TODO: change to ensure that the right sql statements are reported, not just counts checkFn := func(n int) { records := agentServer.GetLatestRecords() - c.Assert(len(records), Equals, n) + require.Len(t, records, n) for _, r := range records { sqlMeta, exist := agentServer.GetSQLMetaByDigestBlocking(r.SqlDigest, time.Second) - c.Assert(exist, IsTrue) - c.Check(sqlMeta.NormalizedSql, Matches, "select.*from.*join.*") + require.True(t, exist) + require.Regexp(t, "select.*from.*join.*", sqlMeta.NormalizedSql) if len(r.PlanDigest) == 0 { continue } plan, exist := agentServer.GetPlanMetaByDigestBlocking(r.PlanDigest, time.Second) - c.Assert(exist, IsTrue) + require.True(t, exist) plan = strings.Replace(plan, "\n", " ", -1) plan = strings.Replace(plan, "\t", " ", -1) - c.Assert(plan, Matches, ".*Join.*Select.*") + require.Regexp(t, ".*Join.*Select.*", plan) } } runWorkload := func(start, end int) context.CancelFunc { ctx, cancel := context.WithCancel(context.Background()) for i := start; i < end; i++ { query := fmt.Sprintf("select /*+ HASH_JOIN(ta, tb) */ * from t%[1]v ta join t%[1]v tb on ta.a=tb.a where ta.b is not null;", i) - go ts.loopExec(ctx, c, func(db *sql.DB) { - dbt := &DBTest{c, db} - rows := dbt.mustQuery(query) + go ts.loopExec(ctx, t, func(db *sql.DB) { + dbt := testkit.NewDBTestKit(t, db) + rows := dbt.MustQuery(query) for rows.Next() { } }) @@ -1927,12 +1593,12 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLAgent(c *C) { agentServer.WaitCollectCnt(1, time.Second*4) checkFn(0) // Test after set agent address and the evict take effect. - dbt.mustExec("set @@global.tidb_top_sql_max_statement_count=5;") + dbt.MustExec("set @@global.tidb_top_sql_max_statement_count=5;") setTopSQLReceiverAddress(agentServer.Address()) agentServer.WaitCollectCnt(1, time.Second*4) checkFn(5) // Test with wrong agent address, the agent server can't receive any record. - dbt.mustExec("set @@global.tidb_top_sql_max_statement_count=8;") + dbt.MustExec("set @@global.tidb_top_sql_max_statement_count=8;") setTopSQLReceiverAddress("127.0.0.1:65530") agentServer.WaitCollectCnt(1, time.Second*4) @@ -1946,7 +1612,7 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLAgent(c *C) { // case 2: agent hangs for a while cancel2 := runWorkload(0, 10) // empty agent address, should not collect records - dbt.mustExec("set @@global.tidb_top_sql_max_statement_count=5;") + dbt.MustExec("set @@global.tidb_top_sql_max_statement_count=5;") setTopSQLReceiverAddress("") agentServer.WaitCollectCnt(1, time.Second*4) checkFn(0) @@ -1982,7 +1648,7 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLAgent(c *C) { agentServer.Stop() // agent server restart agentServer, err = mockTopSQLReporter.StartMockAgentServer() - c.Assert(err, IsNil) + require.NoError(t, err) setTopSQLReceiverAddress(agentServer.Address()) // check result agentServer.WaitCollectCnt(2, time.Second*8) @@ -1990,15 +1656,15 @@ func (ts *tidbTestTopSQLSuite) TestTopSQLAgent(c *C) { cancel5() } -func (ts *tidbTestTopSQLSuite) loopExec(ctx context.Context, c *C, fn func(db *sql.DB)) { +func (ts *tidbTestTopSQLSuite) loopExec(ctx context.Context, t *testing.T, fn func(db *sql.DB)) { db, err := sql.Open("mysql", ts.getDSN()) - c.Assert(err, IsNil, Commentf("Error connecting")) + require.NoError(t, err, "Error connecting") defer func() { err := db.Close() - c.Assert(err, IsNil) + require.NoError(t, err) }() - dbt := &DBTest{c, db} - dbt.mustExec("use topsql;") + dbt := testkit.NewDBTestKit(t, db) + dbt.MustExec("use topsql;") for { select { case <-ctx.Done(): diff --git a/server/util.go b/server/util.go index 6a8cbad8386c5..2a45dce017dc2 100644 --- a/server/util.go +++ b/server/util.go @@ -236,7 +236,7 @@ func dumpBinaryDateTime(data []byte, t types.Time) []byte { func dumpBinaryRow(buffer []byte, columns []*ColumnInfo, row chunk.Row, d *resultEncoder) ([]byte, error) { if d == nil { - d = &resultEncoder{} + d = newResultEncoder(charset.CharsetUTF8MB4) } buffer = append(buffer, mysql.OKHeader) nullBitmapOff := len(buffer) @@ -294,10 +294,10 @@ type resultEncoder struct { // chsName and encoding are unchanged after the initialization from // session variable @@character_set_results. chsName string - encoding charset.Encoding + encoding *charset.Encoding // dataEncoding can be updated to match the column data charset. - dataEncoding charset.Encoding + dataEncoding *charset.Encoding buffer []byte @@ -309,7 +309,7 @@ type resultEncoder struct { func newResultEncoder(chs string) *resultEncoder { return &resultEncoder{ chsName: chs, - encoding: *charset.NewEncoding(chs), + encoding: charset.NewEncoding(chs), buffer: nil, isBinary: chs == charset.CharsetBinary, isNull: len(chs) == 0, @@ -326,7 +326,7 @@ func (d *resultEncoder) updateDataEncoding(chsID uint16) { if err != nil { logutil.BgLogger().Warn("unknown charset ID", zap.Error(err)) } - d.dataEncoding.UpdateEncoding(charset.Formatted(chs)) + d.dataEncoding = charset.NewEncoding(chs) } func (d *resultEncoder) columnTypeInfoCharsetID(info *ColumnInfo) uint16 { @@ -342,15 +342,15 @@ func (d *resultEncoder) columnTypeInfoCharsetID(info *ColumnInfo) uint16 { } func (d *resultEncoder) encodeMeta(src []byte) []byte { - return d.encodeWith(src, &d.encoding) + return d.encodeWith(src, d.encoding) } func (d *resultEncoder) encodeData(src []byte) []byte { if d.isNull || d.isBinary { // Use the column charset to encode. - return d.encodeWith(src, &d.dataEncoding) + return d.encodeWith(src, d.dataEncoding) } - return d.encodeWith(src, &d.encoding) + return d.encodeWith(src, d.encoding) } func (d *resultEncoder) encodeWith(src []byte, enc *charset.Encoding) []byte { @@ -363,7 +363,7 @@ func (d *resultEncoder) encodeWith(src []byte, enc *charset.Encoding) []byte { func dumpTextRow(buffer []byte, columns []*ColumnInfo, row chunk.Row, d *resultEncoder) ([]byte, error) { if d == nil { - d = &resultEncoder{} + d = newResultEncoder(charset.CharsetUTF8MB4) } tmp := make([]byte, 0, 20) for i, col := range columns { diff --git a/server/util_test.go b/server/util_test.go index e0889a5e6958c..35680210b75e4 100644 --- a/server/util_test.go +++ b/server/util_test.go @@ -20,6 +20,7 @@ import ( "time" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" @@ -113,7 +114,7 @@ func TestDumpTextValue(t *testing.T) { Decimal: mysql.NotFixedDec, }} - dp := &resultEncoder{} + dp := newResultEncoder(charset.CharsetUTF8MB4) null := types.NewIntDatum(0) null.SetNull() bs, err := dumpTextRow(nil, columns, chunk.MutRowFromDatums([]types.Datum{null}).ToRow(), dp) diff --git a/session/session.go b/session/session.go index 83c936ccf30b2..2228431007048 100644 --- a/session/session.go +++ b/session/session.go @@ -544,7 +544,7 @@ func (s *session) doCommit(ctx context.Context) error { } s.txn.SetOption(kv.EnableAsyncCommit, sessVars.EnableAsyncCommit) s.txn.SetOption(kv.Enable1PC, sessVars.Enable1PC) - s.txn.SetOption(kv.ResourceGroupTag, sessVars.StmtCtx.GetResourceGroupTag()) + s.txn.SetOption(kv.ResourceGroupTagger, sessVars.StmtCtx.GetResourceGroupTagger()) // priority of the sysvar is lower than `start transaction with causal consistency only` if val := s.txn.GetOption(kv.GuaranteeLinearizability); val == nil || val.(bool) { // We needn't ask the TiKV client to guarantee linearizability for auto-commit transactions @@ -1115,6 +1115,10 @@ func (s *session) GetGlobalSysVar(name string) (string, error) { return sv.Value, nil } } + // It might have been written from an earlier TiDB version, so we should run the validation function + // To normalize the value to be safe for this version of TiDB. This also happens for session scoped + // variables in loadCommonGlobalVariablesIfNeeded -> SetSystemVarWithRelaxedValidation + sysVar = sv.ValidateWithRelaxedValidation(s.GetSessionVars(), sysVar, variable.ScopeGlobal) return sysVar, nil } @@ -3011,7 +3015,7 @@ func (s *session) GetTxnWriteThroughputSLI() *sli.TxnWriteThroughputSLI { return &s.txn.writeSLI } -var _ telemetry.TemporaryTableFeatureChecker = &session{} +var _ telemetry.TemporaryOrCacheTableFeatureChecker = &session{} // TemporaryTableExists is used by the telemetry package to avoid circle dependency. func (s *session) TemporaryTableExists() bool { @@ -3026,6 +3030,19 @@ func (s *session) TemporaryTableExists() bool { return false } +// CachedTableExists is used by the telemetry package to avoid circle dependency. +func (s *session) CachedTableExists() bool { + is := domain.GetDomain(s).InfoSchema() + for _, dbInfo := range is.AllSchemas() { + for _, tbInfo := range is.SchemaTables(dbInfo.Name) { + if tbInfo.Meta().TableCacheStatusType != model.TableCacheStatusDisable { + return true + } + } + } + return false +} + // GetInfoSchema returns snapshotInfoSchema if snapshot schema is set. // Transaction infoschema is returned if inside an explicit txn. // Otherwise the latest infoschema is returned. diff --git a/session/session_test.go b/session/session_test.go index ceca24f509680..3057ef6ae9ab4 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -689,6 +689,15 @@ func (s *testSessionSuite) TestGlobalVarAccessor(c *C) { _, err = tk.Exec("set global time_zone = 'timezone'") c.Assert(err, NotNil) c.Assert(terror.ErrorEqual(err, variable.ErrUnknownTimeZone), IsTrue) + + // Set the global var to a non canonical form of the value + // i.e. implying that it was set from an earlier version of TiDB. + + tk.MustExec(`REPLACE INTO mysql.global_variables (variable_name, variable_value) VALUES ('tidb_enable_noop_functions', '0')`) + domain.GetDomain(tk.Se).NotifyUpdateSysVarCache() // update cache + v, err = se.GetGlobalSysVar("tidb_enable_noop_functions") + c.Assert(err, IsNil) + c.Assert(v, Equals, "OFF") } func (s *testSessionSuite) TestGetSysVariables(c *C) { diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 03c488c883a0c..07cbd89d18e52 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -30,6 +30,8 @@ import ( "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/resourcegrouptag" "github.com/pingcap/tidb/util/tracing" + "github.com/pingcap/tipb/go-tipb" + "github.com/tikv/client-go/v2/tikvrpc" "github.com/tikv/client-go/v2/util" atomic2 "go.uber.org/atomic" "go.uber.org/zap" @@ -171,8 +173,12 @@ type StatementContext struct { // stmtCache is used to store some statement-related values. stmtCache map[StmtCacheKey]interface{} - // resourceGroupTag cache for the current statement resource group tag. - resourceGroupTag atomic.Value + // resourceGroupTagWithRow cache for the current statement resource group tag (with `Row` label). + resourceGroupTagWithRow atomic.Value + // resourceGroupTagWithIndex cache for the current statement resource group tag (with `Index` label). + resourceGroupTagWithIndex atomic.Value + // resourceGroupTagWithUnknown cache for the current statement resource group tag (with `Unknown` label). + resourceGroupTagWithUnknown atomic.Value // Map to store all CTE storages of current SQL. // Will clean up at the end of the execution. CTEStorageMap interface{} @@ -280,19 +286,47 @@ func (sc *StatementContext) GetPlanDigest() (normalized string, planDigest *pars return sc.planNormalized, sc.planDigest } -// GetResourceGroupTag gets the resource group of the statement. -func (sc *StatementContext) GetResourceGroupTag() []byte { - tag, _ := sc.resourceGroupTag.Load().([]byte) - if len(tag) > 0 { - return tag +// GetResourceGroupTagger returns the implementation of tikvrpc.ResourceGroupTagger related to self. +func (sc *StatementContext) GetResourceGroupTagger() tikvrpc.ResourceGroupTagger { + return func(req *tikvrpc.Request) { + req.ResourceGroupTag = sc.GetResourceGroupTagByLabel( + resourcegrouptag.GetResourceGroupLabelByKey(resourcegrouptag.GetFirstKeyFromRequest(req))) + } +} + +// GetResourceGroupTagByLabel gets the resource group of the statement based on the label. +func (sc *StatementContext) GetResourceGroupTagByLabel(label tipb.ResourceGroupTagLabel) []byte { + switch label { + case tipb.ResourceGroupTagLabel_ResourceGroupTagLabelRow: + v := sc.resourceGroupTagWithRow.Load() + if v != nil { + return v.([]byte) + } + case tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex: + v := sc.resourceGroupTagWithIndex.Load() + if v != nil { + return v.([]byte) + } + case tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown: + v := sc.resourceGroupTagWithUnknown.Load() + if v != nil { + return v.([]byte) + } } normalized, sqlDigest := sc.SQLDigest() if len(normalized) == 0 { return nil } - tag = resourcegrouptag.EncodeResourceGroupTag(sqlDigest, sc.planDigest) - sc.resourceGroupTag.Store(tag) - return tag + newTag := resourcegrouptag.EncodeResourceGroupTag(sqlDigest, sc.planDigest, label) + switch label { + case tipb.ResourceGroupTagLabel_ResourceGroupTagLabelRow: + sc.resourceGroupTagWithRow.Store(newTag) + case tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex: + sc.resourceGroupTagWithIndex.Store(newTag) + case tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown: + sc.resourceGroupTagWithUnknown.Store(newTag) + } + return newTag } // SetPlanDigest sets the normalized plan and plan digest. diff --git a/sessionctx/stmtctx/stmtctx_test.go b/sessionctx/stmtctx/stmtctx_test.go index f5bf2cca866be..acfc9c00866d1 100644 --- a/sessionctx/stmtctx/stmtctx_test.go +++ b/sessionctx/stmtctx/stmtctx_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/util/execdetails" + "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/util" ) @@ -90,3 +91,25 @@ func TestStatementContextPushDownFLags(t *testing.T) { require.Equal(t, tt.out, got) } } + +func TestGetResourceGroupTagByLabel(t *testing.T) { + ctx := stmtctx.StatementContext{OriginalSQL: "SELECT * FROM t"} + tagRow := ctx.GetResourceGroupTagByLabel(tipb.ResourceGroupTagLabel_ResourceGroupTagLabelRow) + tagIndex := ctx.GetResourceGroupTagByLabel(tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex) + tagUnknown := ctx.GetResourceGroupTagByLabel(tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown) + tagRow2 := ctx.GetResourceGroupTagByLabel(tipb.ResourceGroupTagLabel_ResourceGroupTagLabelRow) + tagIndex2 := ctx.GetResourceGroupTagByLabel(tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex) + tagUnknown2 := ctx.GetResourceGroupTagByLabel(tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown) + require.NotEmpty(t, tagRow) + require.NotEmpty(t, tagIndex) + require.NotEmpty(t, tagUnknown) + require.NotEmpty(t, tagRow2) + require.NotEmpty(t, tagIndex2) + require.NotEmpty(t, tagUnknown2) + require.Equal(t, &tagRow, &tagRow2) // mem addr + require.Equal(t, &tagIndex, &tagIndex2) + require.Equal(t, &tagUnknown, &tagUnknown2) + require.NotEqual(t, &tagRow, &tagIndex) + require.NotEqual(t, &tagRow, &tagUnknown) + require.NotEqual(t, &tagIndex, &tagUnknown) +} diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index fbec3e758c910..8eb9a9c7670e6 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -59,7 +59,6 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: BigTables, Value: Off, Type: TypeBool}, {Scope: ScopeNone, Name: "skip_external_locking", Value: "1"}, {Scope: ScopeNone, Name: "innodb_sync_array_size", Value: "1"}, - {Scope: ScopeSession, Name: "rand_seed2", Value: ""}, {Scope: ScopeGlobal, Name: ValidatePasswordCheckUserName, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal, Name: ValidatePasswordNumberCount, Value: "1", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64}, {Scope: ScopeSession, Name: "gtid_next", Value: ""}, @@ -275,7 +274,6 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "binlog_gtid_simple_recovery", Value: "1"}, {Scope: ScopeNone, Name: "performance_schema_digests_size", Value: "10000"}, {Scope: ScopeGlobal | ScopeSession, Name: Profiling, Value: Off, Type: TypeBool}, - {Scope: ScopeSession, Name: "rand_seed1", Value: ""}, {Scope: ScopeGlobal, Name: "sha256_password_proxy_users", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: SQLQuoteShowCreate, Value: On, Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: "binlogging_impossible_mode", Value: "IGNORE_ERROR"}, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index c395d98c85356..57d6f75c25902 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -29,6 +29,8 @@ import ( "sync/atomic" "time" + utilMath "github.com/pingcap/tidb/util/math" + "github.com/pingcap/errors" pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" "github.com/pingcap/tidb/config" @@ -955,6 +957,9 @@ type SessionVars struct { curr int8 data [2]stmtctx.StatementContext } + + // Rng stores the rand_seed1 and rand_seed2 for Rand() function + Rng *utilMath.MysqlRng } // InitStatementContext initializes a StatementContext, the object is reused to reduce allocation. @@ -1188,6 +1193,7 @@ func NewSessionVars() *SessionVars { MPPStoreLastFailTime: make(map[string]time.Time), MPPStoreFailTTL: DefTiDBMPPStoreFailTTL, EnablePlacementChecks: DefEnablePlacementCheck, + Rng: utilMath.NewWithTime(), } vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 4d0a77a5cb511..d66eb6c88096f 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -327,6 +327,10 @@ func (sv *SysVar) validateScope(scope ScopeFlag) error { // may be less sophisticated in normalizing values. But errors should be caught and handled, // because otherwise there will be upgrade issues. func (sv *SysVar) ValidateWithRelaxedValidation(vars *SessionVars, value string, scope ScopeFlag) string { + warns := vars.StmtCtx.GetWarnings() + defer func() { + vars.StmtCtx.SetWarnings(warns) // RelaxedVaidation = trim warnings too. + }() normalizedValue, err := sv.validateFromType(vars, value, scope) if err != nil { return normalizedValue @@ -1857,6 +1861,18 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeNone, Name: "version_compile_os", Value: runtime.GOOS}, {Scope: ScopeNone, Name: "version_compile_machine", Value: runtime.GOARCH}, {Scope: ScopeNone, Name: TiDBAllowFunctionForExpressionIndex, ReadOnly: true, Value: collectAllowFuncName4ExpressionIndex()}, + {Scope: ScopeSession, Name: RandSeed1, Type: TypeInt, Value: "0", skipInit: true, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + s.Rng.SetSeed1(uint32(tidbOptPositiveInt32(val, 0))) + return nil + }, GetSession: func(s *SessionVars) (string, error) { + return "0", nil + }}, + {Scope: ScopeSession, Name: RandSeed2, Type: TypeInt, Value: "0", skipInit: true, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + s.Rng.SetSeed2(uint32(tidbOptPositiveInt32(val, 0))) + return nil + }, GetSession: func(s *SessionVars) (string, error) { + return "0", nil + }}, } func collectAllowFuncName4ExpressionIndex() string { @@ -2179,6 +2195,10 @@ const ( Identity = "identity" // TiDBAllowFunctionForExpressionIndex is the name of `TiDBAllowFunctionForExpressionIndex` system variable. TiDBAllowFunctionForExpressionIndex = "tidb_allow_function_for_expression_index" + // RandSeed1 is the name of 'rand_seed1' system variable. + RandSeed1 = "rand_seed1" + // RandSeed2 is the name of 'rand_seed2' system variable. + RandSeed2 = "rand_seed2" ) // GlobalVarAccessor is the interface for accessing global scope system and status variables. diff --git a/statistics/builder.go b/statistics/builder.go index 13edd2457fd75..cff117547f619 100644 --- a/statistics/builder.go +++ b/statistics/builder.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" ) // SortedBuilder is used to build histograms for PK and index. @@ -67,7 +68,7 @@ func (b *SortedBuilder) Iterate(data types.Datum) error { b.hist.NDV = 1 return nil } - cmp, err := b.hist.GetUpper(int(b.bucketIdx)).CompareDatum(b.sc, &data) + cmp, err := b.hist.GetUpper(int(b.bucketIdx)).Compare(b.sc, &data, collate.GetBinaryCollator()) if err != nil { return errors.Trace(err) } @@ -158,7 +159,7 @@ func buildHist(sc *stmtctx.StatementContext, hg *Histogram, samples []*SampleIte hg.AppendBucket(&samples[0].Value, &samples[0].Value, int64(sampleFactor), int64(ndvFactor)) for i := int64(1); i < sampleNum; i++ { corrXYSum += float64(i) * float64(samples[i].Ordinal) - cmp, err := hg.GetUpper(bucketIdx).CompareDatum(sc, &samples[i].Value) + cmp, err := hg.GetUpper(bucketIdx).Compare(sc, &samples[i].Value, collate.GetBinaryCollator()) if err != nil { return 0, errors.Trace(err) } diff --git a/statistics/feedback.go b/statistics/feedback.go index c6806a9c99189..88824afae0e08 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -35,6 +35,7 @@ import ( "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/logutil" "github.com/pingcap/tidb/util/ranger" "go.uber.org/atomic" @@ -330,14 +331,14 @@ func NonOverlappedFeedbacks(sc *stmtctx.StatementContext, fbs []Feedback) ([]Fee // with the previous chosen feedbacks. var existsErr bool sort.Slice(fbs, func(i, j int) bool { - res, err := fbs[i].Upper.CompareDatum(sc, fbs[j].Upper) + res, err := fbs[i].Upper.Compare(sc, fbs[j].Upper, collate.GetBinaryCollator()) if err != nil { existsErr = true } if existsErr || res != 0 { return res < 0 } - res, err = fbs[i].Lower.CompareDatum(sc, fbs[j].Lower) + res, err = fbs[i].Lower.Compare(sc, fbs[j].Lower, collate.GetBinaryCollator()) if err != nil { existsErr = true } @@ -349,7 +350,7 @@ func NonOverlappedFeedbacks(sc *stmtctx.StatementContext, fbs []Feedback) ([]Fee resFBs := make([]Feedback, 0, len(fbs)) previousEnd := &types.Datum{} for _, fb := range fbs { - res, err := previousEnd.CompareDatum(sc, fb.Lower) + res, err := previousEnd.Compare(sc, fb.Lower, collate.GetBinaryCollator()) if err != nil { return fbs, false } @@ -370,14 +371,14 @@ type BucketFeedback struct { // outOfRange checks if the `val` is between `min` and `max`. func outOfRange(sc *stmtctx.StatementContext, min, max, val *types.Datum) (int, error) { - result, err := val.CompareDatum(sc, min) + result, err := val.Compare(sc, min, collate.GetBinaryCollator()) if err != nil { return 0, err } if result < 0 { return result, nil } - result, err = val.CompareDatum(sc, max) + result, err = val.Compare(sc, max, collate.GetBinaryCollator()) if err != nil { return 0, err } @@ -457,7 +458,7 @@ func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*Bucket } bkt.feedback = append(bkt.feedback, fb) // Update the bound if necessary. - res, err := bkt.lower.CompareDatum(nil, fb.Lower) + res, err := bkt.lower.Compare(nil, fb.Lower, collate.GetBinaryCollator()) if err != nil { logutil.BgLogger().Debug("compare datum failed", zap.Any("value1", bkt.lower), zap.Any("value2", fb.Lower), zap.Error(err)) continue @@ -465,7 +466,7 @@ func buildBucketFeedback(h *Histogram, feedback *QueryFeedback) (map[int]*Bucket if res > 0 { bkt.lower = fb.Lower } - res, err = bkt.upper.CompareDatum(nil, fb.Upper) + res, err = bkt.upper.Compare(nil, fb.Upper, collate.GetBinaryCollator()) if err != nil { logutil.BgLogger().Debug("compare datum failed", zap.Any("value1", bkt.upper), zap.Any("value2", fb.Upper), zap.Error(err)) continue @@ -501,7 +502,7 @@ func (b *BucketFeedback) getBoundaries(num int) []types.Datum { total = 1 // Erase the repeat values. for i := 1; i < len(vals); i++ { - cmp, err := vals[total-1].CompareDatum(nil, &vals[i]) + cmp, err := vals[total-1].Compare(nil, &vals[i], collate.GetBinaryCollator()) if err != nil { logutil.BgLogger().Debug("compare datum failed", zap.Any("value1", vals[total-1]), zap.Any("value2", vals[i]), zap.Error(err)) continue diff --git a/statistics/handle/update.go b/statistics/handle/update.go index e847abe23773b..e8672794a2509 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -252,7 +252,8 @@ func (m indexUsageMap) update(tableID int64, indexID int64, value *IndexUsageInf } func (m indexUsageMap) merge(destMap indexUsageMap) { - for id, item := range destMap { + for id := range destMap { + item := destMap[id] m.updateByKey(id, &item) } } diff --git a/statistics/histogram.go b/statistics/histogram.go index 0fdbfe7fd68f4..a61f1d1405f59 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -768,7 +768,7 @@ func MergeHistograms(sc *stmtctx.StatementContext, lh *Histogram, rh *Histogram, } lh.NDV += rh.NDV lLen := lh.Len() - cmp, err := lh.GetUpper(lLen-1).CompareDatum(sc, rh.GetLower(0)) + cmp, err := lh.GetUpper(lLen-1).Compare(sc, rh.GetLower(0), collate.GetBinaryCollator()) if err != nil { return nil, errors.Trace(err) } @@ -1134,7 +1134,7 @@ func (c *Column) GetColumnRowCount(sc *stmtctx.StatementContext, ranges []*range if lowVal.Kind() == types.KindString { lowVal.SetBytes(collate.GetCollator(lowVal.Collation()).Key(lowVal.GetString())) } - cmp, err := lowVal.CompareDatum(sc, &highVal) + cmp, err := lowVal.Compare(sc, &highVal, collate.GetBinaryCollator()) if err != nil { return 0, errors.Trace(err) } @@ -1797,7 +1797,7 @@ func mergeBucketNDV(sc *stmtctx.StatementContext, left *bucket4Merging, right *b res.NDV = left.NDV return &res, nil } - upperCompare, err := right.upper.CompareDatum(sc, left.upper) + upperCompare, err := right.upper.Compare(sc, left.upper, collate.GetBinaryCollator()) if err != nil { return nil, err } @@ -1811,7 +1811,7 @@ func mergeBucketNDV(sc *stmtctx.StatementContext, left *bucket4Merging, right *b // ___left__| // They have the same upper. if upperCompare == 0 { - lowerCompare, err := right.lower.CompareDatum(sc, left.lower) + lowerCompare, err := right.lower.Compare(sc, left.lower, collate.GetBinaryCollator()) if err != nil { return nil, err } @@ -1842,7 +1842,7 @@ func mergeBucketNDV(sc *stmtctx.StatementContext, left *bucket4Merging, right *b // ____right___| // ____left__| // right.upper > left.upper - lowerCompareUpper, err := right.lower.CompareDatum(sc, left.upper) + lowerCompareUpper, err := right.lower.Compare(sc, left.upper, collate.GetBinaryCollator()) if err != nil { return nil, err } @@ -1859,7 +1859,7 @@ func mergeBucketNDV(sc *stmtctx.StatementContext, left *bucket4Merging, right *b return &res, nil } upperRatio := calcFraction4Datums(right.lower, right.upper, left.upper) - lowerCompare, err := right.lower.CompareDatum(sc, left.lower) + lowerCompare, err := right.lower.Compare(sc, left.lower, collate.GetBinaryCollator()) if err != nil { return nil, err } @@ -1911,7 +1911,7 @@ func mergePartitionBuckets(sc *stmtctx.StatementContext, buckets []*bucket4Mergi for i := len(buckets) - 1; i >= 0; i-- { totNDV += buckets[i].NDV res.Count += buckets[i].Count - compare, err := buckets[i].upper.CompareDatum(sc, res.upper) + compare, err := buckets[i].upper.Compare(sc, res.upper, collate.GetBinaryCollator()) if err != nil { return nil, err } @@ -1966,7 +1966,7 @@ func MergePartitionHist2GlobalHist(sc *stmtctx.StatementContext, hists []*Histog minValue = hist.GetLower(0).Clone() continue } - res, err := hist.GetLower(0).CompareDatum(sc, minValue) + res, err := hist.GetLower(0).Compare(sc, minValue, collate.GetBinaryCollator()) if err != nil { return nil, err } @@ -2006,7 +2006,7 @@ func MergePartitionHist2GlobalHist(sc *stmtctx.StatementContext, hists []*Histog minValue = d.Clone() continue } - res, err := d.CompareDatum(sc, minValue) + res, err := d.Compare(sc, minValue, collate.GetBinaryCollator()) if err != nil { return nil, err } @@ -2028,14 +2028,14 @@ func MergePartitionHist2GlobalHist(sc *stmtctx.StatementContext, hists []*Histog var sortError error sort.Slice(buckets, func(i, j int) bool { - res, err := buckets[i].upper.CompareDatum(sc, buckets[j].upper) + res, err := buckets[i].upper.Compare(sc, buckets[j].upper, collate.GetBinaryCollator()) if err != nil { sortError = err } if res != 0 { return res < 0 } - res, err = buckets[i].lower.CompareDatum(sc, buckets[j].lower) + res, err = buckets[i].lower.Compare(sc, buckets[j].lower, collate.GetBinaryCollator()) if err != nil { sortError = err } @@ -2055,7 +2055,7 @@ func MergePartitionHist2GlobalHist(sc *stmtctx.StatementContext, hists []*Histog bucketNDV += buckets[i].NDV if sum >= totCount*bucketCount/expBucketNumber && sum-prevSum >= gBucketCountThreshold { for ; i > 0; i-- { // if the buckets have the same upper, we merge them into the same new buckets. - res, err := buckets[i-1].upper.CompareDatum(sc, buckets[i].upper) + res, err := buckets[i-1].upper.Compare(sc, buckets[i].upper, collate.GetBinaryCollator()) if err != nil { return nil, err } diff --git a/statistics/selectivity.go b/statistics/selectivity.go index 0432e58f47409..7a85404c1e3e8 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -362,13 +362,12 @@ func (coll *HistColl) Selectivity(ctx sessionctx.Context, exprs []expression.Exp } func getMaskAndRanges(ctx sessionctx.Context, exprs []expression.Expression, rangeType ranger.RangeType, lengths []int, cachedPath *planutil.AccessPath, cols ...*expression.Column) (mask int64, ranges []*ranger.Range, partCover bool, err error) { - sc := ctx.GetSessionVars().StmtCtx isDNF := false var accessConds, remainedConds []expression.Expression switch rangeType { case ranger.ColumnRangeType: accessConds = ranger.ExtractAccessConditionsForColumn(exprs, cols[0]) - ranges, err = ranger.BuildColumnRange(accessConds, sc, cols[0].RetType, types.UnspecifiedLength) + ranges, err = ranger.BuildColumnRange(accessConds, ctx, cols[0].RetType, types.UnspecifiedLength) case ranger.IndexRangeType: if cachedPath != nil { ranges, accessConds, remainedConds, isDNF = cachedPath.Ranges, cachedPath.AccessConds, cachedPath.TableFilters, cachedPath.IsDNFCond diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 77ed84366c0a4..e2ab8de16c30b 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -793,14 +793,16 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *backoff.Backo } req := tikvrpc.NewRequest(task.cmdType, &copReq, kvrpcpb.Context{ - IsolationLevel: isolationLevelToPB(b.req.IsolationLevel), - Priority: priorityToPB(b.req.Priority), - NotFillCache: b.req.NotFillCache, - RecordTimeStat: true, - RecordScanStat: true, - TaskId: b.req.TaskID, - ResourceGroupTag: b.req.ResourceGroupTag, + IsolationLevel: isolationLevelToPB(b.req.IsolationLevel), + Priority: priorityToPB(b.req.Priority), + NotFillCache: b.req.NotFillCache, + RecordTimeStat: true, + RecordScanStat: true, + TaskId: b.req.TaskID, }) + if b.req.ResourceGroupTagger != nil { + b.req.ResourceGroupTagger(req) + } req.StoreTp = tikvrpc.TiFlash logutil.BgLogger().Debug("send batch request to ", zap.String("req info", req.String()), zap.Int("cop task len", len(task.regionInfos))) diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 77fbd2a90a6e9..5606cb246e863 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -706,14 +706,16 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch } req := tikvrpc.NewReplicaReadRequest(task.cmdType, &copReq, options.GetTiKVReplicaReadType(worker.req.ReplicaRead), &worker.replicaReadSeed, kvrpcpb.Context{ - IsolationLevel: isolationLevelToPB(worker.req.IsolationLevel), - Priority: priorityToPB(worker.req.Priority), - NotFillCache: worker.req.NotFillCache, - RecordTimeStat: true, - RecordScanStat: true, - TaskId: worker.req.TaskID, - ResourceGroupTag: worker.req.ResourceGroupTag, + IsolationLevel: isolationLevelToPB(worker.req.IsolationLevel), + Priority: priorityToPB(worker.req.Priority), + NotFillCache: worker.req.NotFillCache, + RecordTimeStat: true, + RecordScanStat: true, + TaskId: worker.req.TaskID, }) + if worker.req.ResourceGroupTagger != nil { + worker.req.ResourceGroupTagger(req) + } req.StoreTp = getEndPointType(task.storeType) startTime := time.Now() if worker.kvclient.Stats == nil { diff --git a/store/copr/key_ranges.go b/store/copr/key_ranges.go index 048353df8970b..81e5b22f916aa 100644 --- a/store/copr/key_ranges.go +++ b/store/copr/key_ranges.go @@ -103,7 +103,8 @@ func (r *KeyRanges) Do(f func(ran *kv.KeyRange)) { if r.first != nil { f(r.first) } - for _, ran := range r.mid { + for i := range r.mid { + ran := r.mid[i] f(&ran) } if r.last != nil { diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 98179d4d62d02..3c372bae83725 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/kv" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/driver/options" + "github.com/tikv/client-go/v2/tikvrpc" "github.com/tikv/client-go/v2/txnkv/txnsnapshot" "github.com/tikv/client-go/v2/txnkv/txnutil" ) @@ -113,6 +114,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetMatchStoreLabels(val.([]*metapb.StoreLabel)) case kv.ResourceGroupTag: s.KVSnapshot.SetResourceGroupTag(val.([]byte)) + case kv.ResourceGroupTagger: + s.KVSnapshot.SetResourceGroupTagger(val.(tikvrpc.ResourceGroupTagger)) case kv.ReadReplicaScope: s.KVSnapshot.SetReadReplicaScope(val.(string)) case kv.SnapInterceptor: diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 0c08e9b3f65db..823d33ac88f59 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -31,6 +31,7 @@ import ( tikverr "github.com/tikv/client-go/v2/error" tikvstore "github.com/tikv/client-go/v2/kv" "github.com/tikv/client-go/v2/tikv" + "github.com/tikv/client-go/v2/tikvrpc" "github.com/tikv/client-go/v2/txnkv/txnsnapshot" ) @@ -223,6 +224,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.KVTxn.GetSnapshot().SetMatchStoreLabels(val.([]*metapb.StoreLabel)) case kv.ResourceGroupTag: txn.KVTxn.SetResourceGroupTag(val.([]byte)) + case kv.ResourceGroupTagger: + txn.KVTxn.SetResourceGroupTagger(val.(tikvrpc.ResourceGroupTagger)) case kv.KVFilter: txn.KVTxn.SetKVFilter(val.(tikv.KVFilter)) case kv.SnapInterceptor: diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index 293f57755ed77..e238c85ee2314 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -1891,23 +1891,30 @@ func (w *GCWorker) doGCPlacementRules(dr util.DelRangeTask) (err error) { } } - // Get the partition ID from the job and DelRangeTask. + // Notify PD to drop the placement rules of partition-ids and table-id, even if there may be no placement rules. + var physicalTableIDs []int64 switch historyJob.Type { case model.ActionDropTable, model.ActionTruncateTable: - var physicalTableIDs []int64 var startKey kv.Key if err = historyJob.DecodeArgs(&startKey, &physicalTableIDs); err != nil { return } - // Notify PD to drop the placement rules of partition-ids and table-id, even if there may be no placement rules. physicalTableIDs = append(physicalTableIDs, historyJob.TableID) - bundles := make([]*placement.Bundle, 0, len(physicalTableIDs)) - for _, id := range physicalTableIDs { - bundles = append(bundles, placement.NewBundle(id)) + case model.ActionDropSchema, model.ActionDropTablePartition, model.ActionTruncateTablePartition: + if err = historyJob.DecodeArgs(&physicalTableIDs); err != nil { + return } - err = infosync.PutRuleBundles(context.TODO(), bundles) } - return + + if len(physicalTableIDs) == 0 { + return + } + + bundles := make([]*placement.Bundle, 0, len(physicalTableIDs)) + for _, id := range physicalTableIDs { + bundles = append(bundles, placement.NewBundle(id)) + } + return infosync.PutRuleBundles(context.TODO(), bundles) } func (w *GCWorker) doGCLabelRules(dr util.DelRangeTask) (err error) { diff --git a/store/gcworker/gc_worker_serial_test.go b/store/gcworker/gc_worker_serial_test.go index 65db98dbc88d4..4a50768ff0301 100644 --- a/store/gcworker/gc_worker_serial_test.go +++ b/store/gcworker/gc_worker_serial_test.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" @@ -1649,14 +1650,36 @@ func TestGCPlacementRules(t *testing.T) { s, clean := createGCWorkerSuite(t) defer clean() - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/store/gcworker/mockHistoryJobForGC", "return(1)")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/store/gcworker/mockHistoryJobForGC", "return(10)")) defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/store/gcworker/mockHistoryJobForGC")) }() - dr := util.DelRangeTask{JobID: 1, ElementID: 1} - err := s.gcWorker.doGCPlacementRules(dr) + bundleID := "TiDB_DDL_10" + bundle, err := placement.NewBundleFromOptions(&model.PlacementSettings{ + PrimaryRegion: "r1", + Regions: "r1, r2", + }) + require.NoError(t, err) + bundle.ID = bundleID + + // prepare bundle before gc + require.NoError(t, infosync.PutRuleBundles(context.Background(), []*placement.Bundle{bundle})) + got, err := infosync.GetRuleBundle(context.Background(), bundleID) + require.NoError(t, err) + require.NotNil(t, got) + require.False(t, got.IsEmpty()) + + // do gc + dr := util.DelRangeTask{JobID: 1, ElementID: 10} + err = s.gcWorker.doGCPlacementRules(dr) + require.NoError(t, err) + + // check bundle deleted after gc + got, err = infosync.GetRuleBundle(context.Background(), bundleID) require.NoError(t, err) + require.NotNil(t, got) + require.True(t, got.IsEmpty()) } func TestGCLabelRules(t *testing.T) { diff --git a/store/mockstore/mockcopr/aggregate.go b/store/mockstore/mockcopr/aggregate.go index 0a0dd6d6cfb56..1e7bcc1a9207a 100644 --- a/store/mockstore/mockcopr/aggregate.go +++ b/store/mockstore/mockcopr/aggregate.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" ) type aggCtxsMapper map[string][]*aggregation.AggEvaluateContext @@ -208,6 +209,7 @@ type streamAggExec struct { aggExprs []aggregation.Aggregation aggCtxs []*aggregation.AggEvaluateContext groupByExprs []expression.Expression + groupByCollators []collate.Collator relatedColOffsets []int row []types.Datum tmpGroupByRow []types.Datum @@ -288,7 +290,7 @@ func (e *streamAggExec) meetNewGroup(row [][]byte) (bool, error) { return false, errors.Trace(err) } if matched { - c, err := d.CompareDatum(e.evalCtx.sc, &e.nextGroupByRow[i]) + c, err := d.Compare(e.evalCtx.sc, &e.nextGroupByRow[i], e.groupByCollators[i]) if err != nil { return false, errors.Trace(err) } diff --git a/store/mockstore/mockcopr/cop_handler_dag.go b/store/mockstore/mockcopr/cop_handler_dag.go index 4d71a1caa3994..2fd9d8f73d4b3 100644 --- a/store/mockstore/mockcopr/cop_handler_dag.go +++ b/store/mockstore/mockcopr/cop_handler_dag.go @@ -396,12 +396,17 @@ func (h coprHandler) buildStreamAgg(ctx *dagContext, executor *tipb.Executor) (* for _, agg := range aggs { aggCtxs = append(aggCtxs, agg.CreateContext(ctx.evalCtx.sc)) } + groupByCollators := make([]collate.Collator, 0, len(groupBys)) + for _, expr := range groupBys { + groupByCollators = append(groupByCollators, collate.GetCollator(expr.GetType().Collate)) + } return &streamAggExec{ evalCtx: ctx.evalCtx, aggExprs: aggs, aggCtxs: aggCtxs, groupByExprs: groupBys, + groupByCollators: groupByCollators, currGroupByValues: make([][]byte, 0), relatedColOffsets: relatedColOffsets, row: make([]types.Datum, len(ctx.evalCtx.columnInfos)), diff --git a/store/mockstore/unistore/cophandler/cop_handler_test.go b/store/mockstore/unistore/cophandler/cop_handler_test.go index 3bd39184865cc..4e8462c8be651 100644 --- a/store/mockstore/unistore/cophandler/cop_handler_test.go +++ b/store/mockstore/unistore/cophandler/cop_handler_test.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" @@ -339,10 +340,10 @@ func TestPointGet(t *testing.T) { // verify the returned rows value as input expectedRow := data.rows[handle] - eq, err := returnedRow[0].CompareDatum(nil, &expectedRow[0]) + eq, err := returnedRow[0].Compare(nil, &expectedRow[0], collate.GetBinaryCollator()) require.NoError(t, err) require.Equal(t, 0, eq) - eq, err = returnedRow[1].CompareDatum(nil, &expectedRow[1]) + eq, err = returnedRow[1].Compare(nil, &expectedRow[1], collate.GetBinaryCollator()) require.NoError(t, err) require.Equal(t, 0, eq) } diff --git a/store/mockstore/unistore/cophandler/mpp_exec.go b/store/mockstore/unistore/cophandler/mpp_exec.go index 141e986b068d8..9e2e071513e04 100644 --- a/store/mockstore/unistore/cophandler/mpp_exec.go +++ b/store/mockstore/unistore/cophandler/mpp_exec.go @@ -180,8 +180,8 @@ func (e *exchSenderExec) next() (*chunk.Chunk, error) { } return nil, nil } - for _, tipbChunk := range tipbChunks { - tunnel.DataCh <- &tipbChunk + for j := range tipbChunks { + tunnel.DataCh <- &tipbChunks[j] } } } @@ -194,8 +194,8 @@ func (e *exchSenderExec) next() (*chunk.Chunk, error) { } return nil, nil } - for _, tipbChunk := range tipbChunks { - tunnel.DataCh <- &tipbChunk + for i := range tipbChunks { + tunnel.DataCh <- &tipbChunks[i] } } } diff --git a/table/column.go b/table/column.go index 827087ad1acf2..445a169a82b59 100644 --- a/table/column.go +++ b/table/column.go @@ -24,7 +24,6 @@ import ( "strings" "time" "unicode" - "unicode/utf8" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" @@ -171,9 +170,8 @@ func truncateTrailingSpaces(v *types.Datum) { v.SetString(str, v.Collation()) } -func handleWrongCharsetValue(ctx sessionctx.Context, col *model.ColumnInfo, casted *types.Datum, str string, i int) (types.Datum, error) { +func handleWrongCharsetValue(ctx sessionctx.Context, col *model.ColumnInfo, str string, i int) error { sc := ctx.GetSessionVars().StmtCtx - var strval strings.Builder for j := 0; j < 6; j++ { if len(str) > (i + j) { @@ -187,14 +185,11 @@ func handleWrongCharsetValue(ctx sessionctx.Context, col *model.ColumnInfo, cast if len(str) > i+6 { strval.WriteString(`...`) } - // TODO: Add 'at row %d' err := ErrTruncatedWrongValueForField.FastGen("Incorrect string value '%s' for column '%s'", strval.String(), col.Name) logutil.BgLogger().Error("incorrect string value", zap.Uint64("conn", ctx.GetSessionVars().ConnectionID), zap.Error(err)) - // Truncate to valid utf8 string. - truncateVal := types.NewStringDatum(str[:i]) err = sc.HandleTruncate(err) - return truncateVal, err + return err } func handleZeroDatetime(ctx sessionctx.Context, col *model.ColumnInfo, casted types.Datum, str string, tmIsInvalid bool) (types.Datum, bool, error) { @@ -319,61 +314,48 @@ func CastValue(ctx sessionctx.Context, val types.Datum, col *model.ColumnInfo, r truncateTrailingSpaces(&casted) } - if col.Charset == charset.CharsetASCII { - if ctx.GetSessionVars().SkipASCIICheck { - return casted, nil - } - + if v := makeStringValidator(ctx, col); v != nil { str := casted.GetString() - for i := 0; i < len(str); i++ { - if str[i] > unicode.MaxASCII { - casted, err = handleWrongCharsetValue(ctx, col, &casted, str, i) - break - } + strategy := charset.TruncateStrategyReplace + if val.Collation() == charset.CollationBin { + strategy = charset.TruncateStrategyTrim } - if forceIgnoreTruncate { - err = nil - } - return casted, err - } - - if ctx.GetSessionVars().SkipUTF8Check { - return casted, nil - } - - if !mysql.IsUTF8Charset(col.Charset) { - return casted, nil - } - str := casted.GetString() - utf8Charset := col.Charset == mysql.UTF8Charset - doMB4CharCheck := utf8Charset && config.GetGlobalConfig().CheckMb4ValueInUTF8 - fastCheck := (col.Charset == mysql.UTF8MB4Charset) && utf8.ValidString(str) - if !fastCheck { - // The following check is slow, if we fast check success, we can avoid this. - for i, w := 0, 0; i < len(str); i += w { - runeValue, width := utf8.DecodeRuneInString(str[i:]) - if runeValue == utf8.RuneError { - if strings.HasPrefix(str[i:], string(utf8.RuneError)) { - w = width - continue - } - casted, err = handleWrongCharsetValue(ctx, col, &casted, str, i) - break - } else if width > 3 && doMB4CharCheck { - // Handle non-BMP characters. - casted, err = handleWrongCharsetValue(ctx, col, &casted, str, i) - break - } - w = width + if newStr, invalidPos := v.Truncate(str, strategy); invalidPos >= 0 { + casted = types.NewStringDatum(newStr) + err = handleWrongCharsetValue(ctx, col, str, invalidPos) } } - if forceIgnoreTruncate { err = nil } return casted, err } +func makeStringValidator(ctx sessionctx.Context, col *model.ColumnInfo) charset.StringValidator { + switch col.Charset { + case charset.CharsetASCII: + if ctx.GetSessionVars().SkipASCIICheck { + return nil + } + return charset.StringValidatorASCII{} + case charset.CharsetUTF8: + if ctx.GetSessionVars().SkipUTF8Check { + return nil + } + needCheckMB4 := config.GetGlobalConfig().CheckMb4ValueInUTF8 + return charset.StringValidatorUTF8{IsUTF8MB4: false, CheckMB4ValueInUTF8: needCheckMB4} + case charset.CharsetUTF8MB4: + if ctx.GetSessionVars().SkipUTF8Check { + return nil + } + return charset.StringValidatorUTF8{IsUTF8MB4: true} + case charset.CharsetLatin1, charset.CharsetBinary: + return nil + default: + return charset.StringValidatorOther{Charset: col.Charset} + } +} + // ColDesc describes column information like MySQL desc and show columns do. type ColDesc struct { Field string diff --git a/table/column_test.go b/table/column_test.go index 434e6341a515c..7c860feeae3d6 100644 --- a/table/column_test.go +++ b/table/column_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" ) @@ -126,7 +127,7 @@ func TestHandleBadNull(t *testing.T) { d := types.Datum{} err := col.HandleBadNull(&d, sc) require.NoError(t, err) - cmp, err := d.CompareDatum(sc, &types.Datum{}) + cmp, err := d.Compare(sc, &types.Datum{}, collate.GetBinaryCollator()) require.NoError(t, err) require.Equal(t, 0, cmp) @@ -255,7 +256,7 @@ func TestGetZeroValue(t *testing.T) { colInfo := &model.ColumnInfo{FieldType: *tt.ft} zv := GetZeroValue(colInfo) require.Equal(t, tt.value.Kind(), zv.Kind()) - cmp, err := zv.CompareDatum(sc, &tt.value) + cmp, err := zv.Compare(sc, &tt.value, collate.GetCollator(tt.ft.Collate)) require.NoError(t, err) require.Equal(t, 0, cmp) }) diff --git a/table/table.go b/table/table.go index f39e9b2d9fa8d..c96c77b587d19 100644 --- a/table/table.go +++ b/table/table.go @@ -252,6 +252,8 @@ var MockTableFromMeta func(tableInfo *model.TableInfo) Table type CachedTable interface { Table + Init(renewCh chan func()) error + // TryReadFromCache checks if the cache table is readable. TryReadFromCache(ts uint64) kv.MemBuffer diff --git a/table/tables/cache.go b/table/tables/cache.go index ba8786a65bae3..c7e4cea2e8814 100644 --- a/table/tables/cache.go +++ b/table/tables/cache.go @@ -20,6 +20,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" @@ -29,6 +30,16 @@ import ( "github.com/tikv/client-go/v2/tikv" ) +// RenewLeaseType define the type for renew lease. +type RenewLeaseType int + +const ( + // RenewReadLease means renew read lease. + RenewReadLease RenewLeaseType = iota + 1 + // RenewWriteLease means renew write lease. + RenewWriteLease +) + var ( _ table.Table = &cachedTable{} _ table.CachedTable = &cachedTable{} @@ -38,6 +49,7 @@ type cachedTable struct { TableCommon cacheData atomic.Value handle StateRemote + renewCh chan func() } // cacheData pack the cache data and lease. @@ -72,31 +84,48 @@ func (c *cachedTable) TryReadFromCache(ts uint64) kv.MemBuffer { } data := tmp.(*cacheData) if ts >= data.Start && ts < data.Lease { + leaseTime := oracle.GetTimeFromTS(data.Lease) + nowTime := oracle.GetTimeFromTS(ts) + distance := leaseTime.Sub(nowTime) + // TODO make this configurable in the following PRs + if distance >= 0 && distance <= (1*time.Second) { + c.renewCh <- c.renewLease(ts, RenewReadLease, data) + } return data } return nil } -var mockStateRemote = struct { +// MockStateRemote represents the information of stateRemote. +// Exported it only for testing. +var MockStateRemote = struct { Ch chan remoteTask Data *mockStateRemoteData }{} // NewCachedTable creates a new CachedTable Instance func NewCachedTable(tbl *TableCommon) (table.Table, error) { - if mockStateRemote.Data == nil { - mockStateRemote.Data = newMockStateRemoteData() - mockStateRemote.Ch = make(chan remoteTask, 100) - go mockRemoteService(mockStateRemote.Data, mockStateRemote.Ch) + if MockStateRemote.Data == nil { + MockStateRemote.Data = newMockStateRemoteData() + MockStateRemote.Ch = make(chan remoteTask, 100) + go mockRemoteService(MockStateRemote.Data, MockStateRemote.Ch) } + ret := &cachedTable{ TableCommon: *tbl, - handle: &mockStateRemoteHandle{mockStateRemote.Ch}, + handle: &mockStateRemoteHandle{MockStateRemote.Ch}, + renewCh: make(chan func()), } - return ret, nil } +// Init is an extra operation for cachedTable after TableFromMeta, +// Because cachedTable need some additional parameter that can't be passed in TableFromMeta. +func (c *cachedTable) Init(renewCh chan func()) error { + c.renewCh = renewCh + return nil +} + func (c *cachedTable) loadDataFromOriginalTable(store kv.Storage, lease uint64) (kv.MemBuffer, uint64, error) { buffer, err := newMemBuffer(store) if err != nil { @@ -203,3 +232,21 @@ func (c *cachedTable) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []type } return c.TableCommon.RemoveRecord(ctx, h, r) } + +func (c *cachedTable) renewLease(ts uint64, op RenewLeaseType, data *cacheData) func() { + return func() { + tid := c.Meta().ID + lease := leaseFromTS(ts) + succ, err := c.handle.RenewLease(tid, ts, lease, op) + if err != nil { + log.Warn("Renew read lease error") + } + if succ { + c.cacheData.Store(&cacheData{ + Start: data.Start, + Lease: lease, + MemBuffer: data, + }) + } + } +} diff --git a/table/tables/cache_test.go b/table/tables/cache_test.go index e1c55ab3d97af..a435f10422e85 100644 --- a/table/tables/cache_test.go +++ b/table/tables/cache_test.go @@ -18,6 +18,9 @@ import ( "testing" "time" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" ) @@ -358,7 +361,7 @@ func TestCacheTableBatchPointGet(t *testing.T) { 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)") - + tk.MustExec("alter table bp_cache_tmp1 cache") // 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")) @@ -389,3 +392,28 @@ func TestCacheTableBatchPointGet(t *testing.T) { 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")) } + +func TestRenewLease(t *testing.T) { + // Test RenewLeaseForRead + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + se := tk.Session() + tk.MustExec("create table cache_renew_t (id int)") + tk.MustExec("alter table cache_renew_t cache") + tbl, err := se.GetInfoSchema().(infoschema.InfoSchema).TableByName(model.NewCIStr("test"), model.NewCIStr("cache_renew_t")) + require.NoError(t, err) + var i int + tk.MustExec("select * from cache_renew_t") + _, oldLease, _ := tables.MockStateRemote.Data.Load(tbl.Meta().ID) + for i = 0; i < 20; i++ { + time.Sleep(200 * time.Millisecond) + tk.MustExec("select * from cache_renew_t") + _, lease, _ := tables.MockStateRemote.Data.Load(tbl.Meta().ID) + if lease != oldLease { + break + } + } + require.True(t, i < 20) +} diff --git a/table/tables/state_remote.go b/table/tables/state_remote.go index 5ef43271955b7..46a304127ec5e 100644 --- a/table/tables/state_remote.go +++ b/table/tables/state_remote.go @@ -48,8 +48,8 @@ type StateRemote interface { // LockForWrite try to add a write lock to the table with the specified tableID LockForWrite(tid int64, now, ts uint64) error - // RenewLease attempt to renew the read lock on the table with the specified tableID - RenewLease(tid int64, ts uint64) (bool, error) + // RenewLease attempt to renew the read / write lock on the table with the specified tableID + RenewLease(tid int64, oldTs uint64, newTs uint64, op RenewLeaseType) (bool, error) } // mockStateRemoteHandle implement the StateRemote interface. @@ -101,7 +101,17 @@ func (r *mockStateRemoteHandle) LockForWrite(tid int64, now, ts uint64) error { return op.err } -func (r *mockStateRemoteHandle) RenewLease(tid int64, ts uint64) (bool, error) { +func (r *mockStateRemoteHandle) RenewLease(tid int64, oldTs uint64, newTs uint64, op RenewLeaseType) (bool, error) { + switch op { + case RenewReadLease: + op := &renewLeaseForReadOP{tid: tid, oldTs: oldTs, newTs: newTs} + op.Add(1) + r.ch <- op + op.Wait() + return op.succ, op.err + case RenewWriteLease: + // TODO : Renew Write Lease will implement in next pr. + } return false, errors.New("not implemented yet") } @@ -168,6 +178,24 @@ func (op *lockForWriteOP) Exec(data *mockStateRemoteData) { op.Done() } +// renewForReadOP is a kind of remote task +type renewLeaseForReadOP struct { + sync.WaitGroup + // Input + tid int64 + oldTs uint64 + newTs uint64 + + // Output + succ bool + err error +} + +func (op *renewLeaseForReadOP) Exec(r *mockStateRemoteData) { + op.succ, op.err = r.renewLeaseForRead(op.tid, op.oldTs, op.newTs) + op.Done() +} + type mockStateRemoteData struct { data map[int64]*stateRecord } @@ -276,3 +304,27 @@ func (r *mockStateRemoteData) LockForWrite(tid int64, now, ts uint64) (uint64, e } return 0, nil } + +func (r *mockStateRemoteData) renewLeaseForRead(tid int64, oldTs uint64, newTs uint64) (bool, error) { + record, ok := r.data[tid] + if !ok { + record = &stateRecord{ + lockLease: newTs, + lockType: CachedTableLockRead, + } + r.data[tid] = record + return true, nil + } + if record.lockType != CachedTableLockRead { + return false, errors.New("The read lock can be renewed only in the read lock state") + } + if record.lockLease < oldTs { + return false, errors.New("The remote Lease is invalid") + } + if record.lockLease <= newTs { + record.lockLease = newTs + return true, nil + } + return false, errors.New("The new lease is smaller than the old lease is an illegal contract renewal operation") + +} diff --git a/table/tables/tables.go b/table/tables/tables.go index ecb7e23137745..5adf626ca2f78 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -84,6 +84,13 @@ func MockTableFromMeta(tblInfo *model.TableInfo) table.Table { var t TableCommon initTableCommon(&t, tblInfo, tblInfo.ID, columns, nil) + if tblInfo.TableCacheStatusType != model.TableCacheStatusDisable { + ret, err := NewCachedTable(&t) + if err != nil { + return nil + } + return ret + } if tblInfo.GetPartitionInfo() == nil { if err := initTableIndices(&t); err != nil { return nil diff --git a/tablecodec/rowindexcodec/main_test.go b/tablecodec/rowindexcodec/main_test.go new file mode 100644 index 0000000000000..55b15ba96e15d --- /dev/null +++ b/tablecodec/rowindexcodec/main_test.go @@ -0,0 +1,27 @@ +// 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 rowindexcodec + +import ( + "testing" + + "github.com/pingcap/tidb/util/testbridge" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testbridge.WorkaroundGoCheckFlags() + goleak.VerifyTestMain(m) +} diff --git a/tablecodec/rowindexcodec/rowindexcodec.go b/tablecodec/rowindexcodec/rowindexcodec.go new file mode 100644 index 0000000000000..924d3fff2e5cf --- /dev/null +++ b/tablecodec/rowindexcodec/rowindexcodec.go @@ -0,0 +1,57 @@ +// 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 rowindexcodec + +import ( + "bytes" +) + +// KeyKind is a specific type of key, mainly used to distinguish row/index. +type KeyKind int + +const ( + // KeyKindUnknown indicates that this key is unknown type. + KeyKindUnknown KeyKind = iota + // KeyKindRow means that this key belongs to row. + KeyKindRow + // KeyKindIndex means that this key belongs to index. + KeyKindIndex +) + +var ( + tablePrefix = []byte{'t'} + rowPrefix = []byte("_r") + indexPrefix = []byte("_i") +) + +// GetKeyKind returns the KeyKind that matches the key in the minimalist way. +func GetKeyKind(key []byte) KeyKind { + // [ TABLE_PREFIX | TABLE_ID | ROW_PREFIX (INDEX_PREFIX) | ROW_ID (INDEX_ID) | ... ] (name) + // [ 1 | 8 | 2 | 8 | ... ] (byte) + if len(key) < 11 { + return KeyKindUnknown + } + if !bytes.HasPrefix(key, tablePrefix) { + return KeyKindUnknown + } + key = key[9:] + if bytes.HasPrefix(key, rowPrefix) { + return KeyKindRow + } + if bytes.HasPrefix(key, indexPrefix) { + return KeyKindIndex + } + return KeyKindUnknown +} diff --git a/tablecodec/rowindexcodec/rowindexcodec_test.go b/tablecodec/rowindexcodec/rowindexcodec_test.go new file mode 100644 index 0000000000000..146fc80d9bee0 --- /dev/null +++ b/tablecodec/rowindexcodec/rowindexcodec_test.go @@ -0,0 +1,28 @@ +// 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 rowindexcodec + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestGetKeyKind(t *testing.T) { + require.Equal(t, KeyKindRow, GetKeyKind([]byte{116, 128, 0, 0, 0, 0, 0, 0, 0, 95, 114})) + require.Equal(t, KeyKindIndex, GetKeyKind([]byte{116, 128, 0, 0, 0, 0, 0, 0, 0, 95, 105, 128, 0, 0, 0, 0, 0, 0, 0})) + require.Equal(t, KeyKindUnknown, GetKeyKind([]byte(""))) + require.Equal(t, KeyKindUnknown, GetKeyKind(nil)) +} diff --git a/tablecodec/tablecodec_test.go b/tablecodec/tablecodec_test.go index 554287b8dc6a7..e373eadba33b9 100644 --- a/tablecodec/tablecodec_test.go +++ b/tablecodec/tablecodec_test.go @@ -46,6 +46,20 @@ func TestTableCodec(t *testing.T) { require.Equal(t, int64(2), h.IntValue()) } +// https://github.com/pingcap/tidb/issues/27687. +func TestTableCodecInvalid(t *testing.T) { + tableID := int64(100) + buf := make([]byte, 0, 11) + buf = append(buf, 't') + buf = codec.EncodeInt(buf, tableID) + buf = append(buf, '_', 'r') + buf = codec.EncodeInt(buf, -9078412423848787968) + buf = append(buf, '0') + _, err := DecodeRowKey(buf) + require.NotNil(t, err) + require.Equal(t, "invalid encoded key", err.Error()) +} + // column is a structure used for test type column struct { id int64 diff --git a/telemetry/data_feature_usage.go b/telemetry/data_feature_usage.go index 0459fb9104849..cf32510097423 100644 --- a/telemetry/data_feature_usage.go +++ b/telemetry/data_feature_usage.go @@ -36,6 +36,7 @@ type featureUsage struct { ClusterIndex *ClusterIndexUsage `json:"clusterIndex"` TemporaryTable bool `json:"temporaryTable"` CTE *m.CTEUsageCounter `json:"cte"` + CachedTable bool `json:"cachedTable"` } func getFeatureUsage(ctx sessionctx.Context) (*featureUsage, error) { @@ -49,11 +50,13 @@ func getFeatureUsage(ctx sessionctx.Context) (*featureUsage, error) { txnUsage := getTxnUsageInfo(ctx) // Avoid the circle dependency. - temporaryTable := ctx.(TemporaryTableFeatureChecker).TemporaryTableExists() + temporaryTable := ctx.(TemporaryOrCacheTableFeatureChecker).TemporaryTableExists() cteUsage := getCTEUsageInfo() - return &featureUsage{txnUsage, clusterIdxUsage, temporaryTable, cteUsage}, nil + cachedTable := ctx.(TemporaryOrCacheTableFeatureChecker).CachedTableExists() + + return &featureUsage{txnUsage, clusterIdxUsage, temporaryTable, cteUsage, cachedTable}, nil } // ClusterIndexUsage records the usage info of all the tables, no more than 10k tables @@ -138,10 +141,11 @@ func getClusterIndexUsageInfo(ctx sessionctx.Context) (cu *ClusterIndexUsage, er return &usage, nil } -// TemporaryTableFeatureChecker is defined to avoid package circle dependency. +// TemporaryOrCacheTableFeatureChecker is defined to avoid package circle dependency. // The session struct implements this interface. -type TemporaryTableFeatureChecker interface { +type TemporaryOrCacheTableFeatureChecker interface { TemporaryTableExists() bool + CachedTableExists() bool } // TxnUsage records the usage info of transaction related features, including diff --git a/telemetry/data_feature_usage_test.go b/telemetry/data_feature_usage_test.go index 932a85127f4bd..8f5536416811c 100644 --- a/telemetry/data_feature_usage_test.go +++ b/telemetry/data_feature_usage_test.go @@ -87,3 +87,27 @@ func TestTemporaryTable(t *testing.T) { require.NoError(t, err) require.True(t, usage.TemporaryTable) } + +func TestCachedTable(t *testing.T) { + t.Parallel() + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + usage, err := telemetry.GetFeatureUsage(tk.Session()) + require.NoError(t, err) + require.False(t, usage.CachedTable) + tk.MustExec("drop table if exists tele_cache_t") + tk.MustExec("create table tele_cache_t (id int)") + tk.MustExec("alter table tele_cache_t cache") + usage, err = telemetry.GetFeatureUsage(tk.Session()) + require.NoError(t, err) + require.True(t, usage.CachedTable) + tk.MustExec("alter table tele_cache_t nocache") + usage, err = telemetry.GetFeatureUsage(tk.Session()) + require.NoError(t, err) + require.False(t, usage.CachedTable) +} diff --git a/telemetry/main_test.go b/telemetry/main_test.go index f498d16a2d564..3ab35518fd644 100644 --- a/telemetry/main_test.go +++ b/telemetry/main_test.go @@ -32,6 +32,7 @@ func TestMain(m *testing.M) { opts := []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + goleak.IgnoreTopFunction("github.com/pingcap/tidb/table/tables.mockRemoteService"), } goleak.VerifyTestMain(m, opts...) diff --git a/testkit/dbtestkit.go b/testkit/dbtestkit.go index f7d6de4d07244..5682597618683 100644 --- a/testkit/dbtestkit.go +++ b/testkit/dbtestkit.go @@ -42,6 +42,29 @@ func NewDBTestKit(t *testing.T, db *sql.DB) *DBTestKit { } } +// MustPrepare creates a prepared statement for later queries or executions. +func (tk *DBTestKit) MustPrepare(query string) *sql.Stmt { + stmt, err := tk.db.Prepare(query) + tk.require.NoErrorf(err, "Prepare %s", query) + return stmt +} + +// MustExecPrepared executes a prepared statement with the given arguments and +// returns a Result summarizing the effect of the statement. +func (tk *DBTestKit) MustExecPrepared(stmt *sql.Stmt, args ...interface{}) sql.Result { + res, err := stmt.Exec(args...) + tk.require.NoErrorf(err, "Execute prepared with args: %s", args) + return res +} + +// MustQueryPrepared executes a prepared query statement with the given arguments +// and returns the query results as a *Rows. +func (tk *DBTestKit) MustQueryPrepared(stmt *sql.Stmt, args ...interface{}) *sql.Rows { + rows, err := stmt.Query(args...) + tk.require.NoErrorf(err, "Query prepared with args: %s", args) + return rows +} + // MustExec query the statements and returns the result. func (tk *DBTestKit) MustExec(sql string, args ...interface{}) sql.Result { comment := fmt.Sprintf("sql:%s, args:%v", sql, args) @@ -60,6 +83,14 @@ func (tk *DBTestKit) MustQuery(sql string, args ...interface{}) *sql.Rows { return rows } +// MustQueryRows query the statements +func (tk *DBTestKit) MustQueryRows(query string, args ...interface{}) { + rows := tk.MustQuery(query, args...) + tk.require.True(rows.Next()) + tk.require.NoError(rows.Err()) + rows.Close() +} + // GetDB returns the underlay sql.DB instance. func (tk *DBTestKit) GetDB() *sql.DB { return tk.db diff --git a/testkit/testkit.go b/testkit/testkit.go index a94cd076e86fc..50af5e0178a1b 100644 --- a/testkit/testkit.go +++ b/testkit/testkit.go @@ -228,6 +228,12 @@ func (tk *TestKit) MustUseIndex(sql string, index string, args ...interface{}) b return false } +// CheckExecResult checks the affected rows and the insert id after executing MustExec. +func (tk *TestKit) CheckExecResult(affectedRows, insertID int64) { + tk.require.Equal(int64(tk.Session().AffectedRows()), affectedRows) + tk.require.Equal(int64(tk.Session().LastInsertID()), insertID) +} + // WithPruneMode run test case under prune mode. func WithPruneMode(tk *TestKit, mode variable.PartitionPruneMode, f func()) { tk.MustExec("set @@tidb_partition_prune_mode=`" + string(mode) + "`") diff --git a/types/convert_test.go b/types/convert_test.go index 125c3b061fabf..2e5b3f1a1d91e 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -240,10 +240,10 @@ func TestConvertType(t *testing.T) { require.Truef(t, terror.ErrorEqual(err, ErrOverflow), "err %v", err) require.Equal(t, "-9999.9999", v.(*MyDecimal).String()) v, err = Convert("1,999.00", ft) - require.Truef(t, terror.ErrorEqual(err, ErrBadNumber), "err %v", err) + require.Truef(t, terror.ErrorEqual(err, ErrTruncated), "err %v", err) require.Equal(t, "1.0000", v.(*MyDecimal).String()) v, err = Convert("1,999,999.00", ft) - require.Truef(t, terror.ErrorEqual(err, ErrBadNumber), "err %v", err) + require.Truef(t, terror.ErrorEqual(err, ErrTruncated), "err %v", err) require.Equal(t, "1.0000", v.(*MyDecimal).String()) v, err = Convert("199.00 ", ft) require.NoError(t, err) diff --git a/types/datum.go b/types/datum.go index d79a086cd7346..03874e1f02e35 100644 --- a/types/datum.go +++ b/types/datum.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/parser/types" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types/json" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/hack" ) @@ -366,6 +367,7 @@ func (d *Datum) GetRaw() []byte { } // SetAutoID set the auto increment ID according to its int flag. +// Don't use it directly, useless wrapped with setDatumAutoIDAndCast. func (d *Datum) SetAutoID(id int64, flag uint) { if mysql.HasUnsignedFlag(flag) { d.SetUint64(uint64(id)) @@ -548,7 +550,63 @@ func (d *Datum) SetValue(val interface{}, tp *types.FieldType) { } } +// Compare compares datum to another datum. +// Notes: don't rely on datum.collation to get the collator, it's tend to buggy. +// TODO: use this function to replace CompareDatum. After we remove all of usage of CompareDatum, we can rename this function back to CompareDatum. +func (d *Datum) Compare(sc *stmtctx.StatementContext, ad *Datum, comparer collate.Collator) (int, error) { + if d.k == KindMysqlJSON && ad.k != KindMysqlJSON { + cmp, err := ad.Compare(sc, d, comparer) + return cmp * -1, errors.Trace(err) + } + switch ad.k { + case KindNull: + if d.k == KindNull { + return 0, nil + } + return 1, nil + case KindMinNotNull: + if d.k == KindNull { + return -1, nil + } else if d.k == KindMinNotNull { + return 0, nil + } + return 1, nil + case KindMaxValue: + if d.k == KindMaxValue { + return 0, nil + } + return -1, nil + case KindInt64: + return d.compareInt64(sc, ad.GetInt64()) + case KindUint64: + return d.compareUint64(sc, ad.GetUint64()) + case KindFloat32, KindFloat64: + return d.compareFloat64(sc, ad.GetFloat64()) + case KindString: + return d.compareStringNew(sc, ad.GetString(), comparer) + case KindBytes: + return comparer.Compare(d.GetString(), ad.GetString()), nil + case KindMysqlDecimal: + return d.compareMysqlDecimal(sc, ad.GetMysqlDecimal()) + case KindMysqlDuration: + return d.compareMysqlDuration(sc, ad.GetMysqlDuration()) + case KindMysqlEnum: + return d.compareMysqlEnumNew(sc, ad.GetMysqlEnum(), comparer) + case KindBinaryLiteral, KindMysqlBit: + return d.compareBinaryLiteralNew(sc, ad.GetBinaryLiteral4Cmp(), comparer) + case KindMysqlSet: + return d.compareMysqlSetNew(sc, ad.GetMysqlSet(), comparer) + case KindMysqlJSON: + return d.compareMysqlJSON(sc, ad.GetMysqlJSON()) + case KindMysqlTime: + return d.compareMysqlTime(sc, ad.GetMysqlTime()) + default: + return 0, nil + } +} + // CompareDatum compares datum to another datum. +// Deprecated: will be replaced with Compare. // TODO: return error properly. func (d *Datum) CompareDatum(sc *stmtctx.StatementContext, ad *Datum) (int, error) { if d.k == KindMysqlJSON && ad.k != KindMysqlJSON { @@ -673,6 +731,39 @@ func (d *Datum) compareFloat64(sc *stmtctx.StatementContext, f float64) (int, er } } +func (d *Datum) compareStringNew(sc *stmtctx.StatementContext, s string, comparer collate.Collator) (int, error) { + switch d.k { + case KindNull, KindMinNotNull: + return -1, nil + case KindMaxValue: + return 1, nil + case KindString, KindBytes: + return comparer.Compare(d.GetString(), s), nil + case KindMysqlDecimal: + dec := new(MyDecimal) + err := sc.HandleTruncate(dec.FromString(hack.Slice(s))) + return d.GetMysqlDecimal().Compare(dec), errors.Trace(err) + case KindMysqlTime: + dt, err := ParseDatetime(sc, s) + return d.GetMysqlTime().Compare(dt), errors.Trace(err) + case KindMysqlDuration: + dur, err := ParseDuration(sc, s, MaxFsp) + return d.GetMysqlDuration().Compare(dur), errors.Trace(err) + case KindMysqlSet: + return comparer.Compare(d.GetMysqlSet().String(), s), nil + case KindMysqlEnum: + return comparer.Compare(d.GetMysqlEnum().String(), s), nil + case KindBinaryLiteral, KindMysqlBit: + return comparer.Compare(d.GetBinaryLiteral4Cmp().String(), s), nil + default: + fVal, err := StrToFloat(sc, s, false) + if err != nil { + return 0, errors.Trace(err) + } + return d.compareFloat64(sc, fVal) + } +} + func (d *Datum) compareString(sc *stmtctx.StatementContext, s string, retCollation string) (int, error) { switch d.k { case KindNull, KindMinNotNull: @@ -748,6 +839,52 @@ func (d *Datum) compareMysqlDuration(sc *stmtctx.StatementContext, dur Duration) } } +func (d *Datum) compareMysqlEnumNew(sc *stmtctx.StatementContext, enum Enum, comparer collate.Collator) (int, error) { + switch d.k { + case KindNull, KindMinNotNull: + return -1, nil + case KindMaxValue: + return 1, nil + case KindString, KindBytes, KindMysqlEnum, KindMysqlSet: + return comparer.Compare(d.GetString(), enum.String()), nil + default: + return d.compareFloat64(sc, enum.ToNumber()) + } +} + +func (d *Datum) compareBinaryLiteralNew(sc *stmtctx.StatementContext, b BinaryLiteral, comparer collate.Collator) (int, error) { + switch d.k { + case KindNull, KindMinNotNull: + return -1, nil + case KindMaxValue: + return 1, nil + case KindString, KindBytes: + fallthrough // in this case, d is converted to Binary and then compared with b + case KindBinaryLiteral, KindMysqlBit: + return comparer.Compare(d.GetBinaryLiteral4Cmp().ToString(), b.ToString()), nil + default: + val, err := b.ToInt(sc) + if err != nil { + return 0, errors.Trace(err) + } + result, err := d.compareFloat64(sc, float64(val)) + return result, errors.Trace(err) + } +} + +func (d *Datum) compareMysqlSetNew(sc *stmtctx.StatementContext, set Set, comparer collate.Collator) (int, error) { + switch d.k { + case KindNull, KindMinNotNull: + return -1, nil + case KindMaxValue: + return 1, nil + case KindString, KindBytes, KindMysqlEnum, KindMysqlSet: + return comparer.Compare(d.GetString(), set.String()), nil + default: + return d.compareFloat64(sc, set.ToNumber()) + } +} + func (d *Datum) compareMysqlEnum(sc *stmtctx.StatementContext, enum Enum) (int, error) { switch d.k { case KindNull, KindMinNotNull: @@ -996,9 +1133,9 @@ func ProduceStrWithSpecifiedTp(s string, tp *FieldType, sc *stmtctx.StatementCon // overflowed part is all whitespaces var overflowed string var characterLen int - // Flen is the rune length, not binary length, for UTF8 charset, we need to calculate the + // Flen is the rune length, not binary length, for Non-binary charset, we need to calculate the // rune count and truncate to Flen runes if it is too long. - if chs == charset.CharsetUTF8 || chs == charset.CharsetUTF8MB4 { + if chs != charset.CharsetBinary { characterLen = utf8.RuneCountInString(s) if characterLen > flen { // 1. If len(s) is 0 and flen is 0, truncateLen will be 0, don't truncate s. @@ -1374,7 +1511,7 @@ func ProduceDecWithSpecifiedTp(dec *MyDecimal, tp *FieldType, sc *stmtctx.Statem if err != nil { return nil, err } - if !dec.IsZero() && frac > decimal && dec.Compare(&old) != 0 { + if !old.IsZero() && frac > decimal && dec.Compare(&old) != 0 { sc.AppendWarning(ErrTruncatedWrongVal.GenWithStackByArgs("DECIMAL", &old)) err = nil } @@ -1499,7 +1636,13 @@ func (d *Datum) convertToMysqlEnum(sc *stmtctx.StatementContext, target *FieldTy case KindString, KindBytes, KindBinaryLiteral: e, err = ParseEnum(target.Elems, d.GetString(), target.Collate) case KindMysqlEnum: - e, err = ParseEnum(target.Elems, d.GetMysqlEnum().Name, target.Collate) + if d.i == 0 { + // MySQL enum zero value has an empty string name(Enum{Name: '', Value: 0}). It is + // different from the normal enum string value(Enum{Name: '', Value: n}, n > 0). + e = Enum{} + } else { + e, err = ParseEnum(target.Elems, d.GetMysqlEnum().Name, target.Collate) + } case KindMysqlSet: e, err = ParseEnum(target.Elems, d.GetMysqlSet().Name, target.Collate) default: @@ -1507,11 +1650,10 @@ func (d *Datum) convertToMysqlEnum(sc *stmtctx.StatementContext, target *FieldTy uintDatum, err = d.convertToUint(sc, target) if err == nil { e, err = ParseEnumValue(target.Elems, uintDatum.GetUint64()) + } else { + err = errors.Wrap(ErrTruncated, "convert to MySQL enum failed: "+err.Error()) } } - if err != nil { - err = errors.Wrap(ErrTruncated, "convert to MySQL enum failed: "+err.Error()) - } ret.SetMysqlEnum(e, target.Collate) return ret, err } diff --git a/types/enum.go b/types/enum.go index d21c63ab18c7b..47e4684536412 100644 --- a/types/enum.go +++ b/types/enum.go @@ -15,6 +15,7 @@ package types import ( + "fmt" "strconv" "github.com/pingcap/errors" @@ -55,8 +56,8 @@ func ParseEnum(elems []string, name string, collation string) (Enum, error) { if num, err := strconv.ParseUint(name, 0, 64); err == nil { return ParseEnumValue(elems, num) } - - return Enum{}, errors.Errorf("item %s is not in enum %v", name, elems) + errMsg := fmt.Sprintf("convert to MySQL enum failed: item %s is not in enum %v", name, elems) + return Enum{}, errors.Wrap(ErrTruncated, errMsg) } // ParseEnumName creates a Enum with item name. @@ -67,14 +68,15 @@ func ParseEnumName(elems []string, name string, collation string) (Enum, error) return Enum{Name: n, Value: uint64(i) + 1}, nil } } - - return Enum{}, errors.Errorf("item %s is not in enum %v", name, elems) + errMsg := fmt.Sprintf("convert to MySQL enum failed: item %s is not in enum %v", name, elems) + return Enum{}, errors.Wrap(ErrTruncated, errMsg) } // ParseEnumValue creates a Enum with special number. func ParseEnumValue(elems []string, number uint64) (Enum, error) { if number == 0 || number > uint64(len(elems)) { - return Enum{}, errors.Errorf("number %d overflow enum boundary [1, %d]", number, len(elems)) + errMsg := fmt.Sprintf("convert to MySQL enum failed: number %d overflow enum boundary [1, %d]", number, len(elems)) + return Enum{}, errors.Wrap(ErrTruncated, errMsg) } return Enum{Name: elems[number-1], Value: number}, nil diff --git a/types/mydecimal.go b/types/mydecimal.go index d98d64ad09ea6..e0cbe3a928aed 100644 --- a/types/mydecimal.go +++ b/types/mydecimal.go @@ -17,6 +17,7 @@ package types import ( "math" "strconv" + "strings" "github.com/pingcap/errors" "github.com/pingcap/log" @@ -393,8 +394,6 @@ func (d *MyDecimal) ToString() (str []byte) { // FromString parses decimal from string. func (d *MyDecimal) FromString(str []byte) error { - // strErr is used to check str is bad number or not - var strErr error for i := 0; i < len(str); i++ { if !isSpace(str[i]) { str = str[i:] @@ -425,8 +424,6 @@ func (d *MyDecimal) FromString(str []byte) error { endIdx++ } digitsFrac = endIdx - strIdx - 1 - } else if strIdx < len(str) && (str[strIdx] != 'e' && str[strIdx] != 'E' && str[strIdx] != ' ') { - strErr = ErrBadNumber } else { digitsFrac = 0 endIdx = strIdx @@ -486,33 +483,40 @@ func (d *MyDecimal) FromString(str []byte) error { if innerIdx != 0 { d.wordBuf[wordIdx] = word * powers10[digitsPerWord-innerIdx] } - if endIdx+1 <= len(str) && (str[endIdx] == 'e' || str[endIdx] == 'E') { - exponent, err1 := strToInt(string(str[endIdx+1:])) - if err1 != nil { - err = errors.Cause(err1) - if err != ErrTruncated { + if endIdx+1 <= len(str) { + if str[endIdx] == 'e' || str[endIdx] == 'E' { + exponent, err1 := strToInt(string(str[endIdx+1:])) + if err1 != nil { + err = errors.Cause(err1) + if err != ErrTruncated { + *d = zeroMyDecimal + } + } + if exponent > math.MaxInt32/2 { + negative := d.negative + maxDecimal(wordBufLen*digitsPerWord, 0, d) + d.negative = negative + err = ErrOverflow + } + if exponent < math.MinInt32/2 && err != ErrOverflow { *d = zeroMyDecimal + err = ErrTruncated } - } - if exponent > math.MaxInt32/2 { - negative := d.negative - maxDecimal(wordBufLen*digitsPerWord, 0, d) - d.negative = negative - err = ErrOverflow - } - if exponent < math.MinInt32/2 && err != ErrOverflow { - *d = zeroMyDecimal - err = ErrTruncated - } - if err != ErrOverflow { - shiftErr := d.Shift(int(exponent)) - if shiftErr != nil { - if shiftErr == ErrOverflow { - negative := d.negative - maxDecimal(wordBufLen*digitsPerWord, 0, d) - d.negative = negative + if err != ErrOverflow { + shiftErr := d.Shift(int(exponent)) + if shiftErr != nil { + if shiftErr == ErrOverflow { + negative := d.negative + maxDecimal(wordBufLen*digitsPerWord, 0, d) + d.negative = negative + } + err = shiftErr } - err = shiftErr + } + } else { + trimstr := strings.TrimSpace(string(str[endIdx:])) + if len(trimstr) != 0 { + err = ErrTruncated } } } @@ -527,9 +531,6 @@ func (d *MyDecimal) FromString(str []byte) error { d.negative = false } d.resultFrac = d.digitsFrac - if strErr != nil { - return strErr - } return err } diff --git a/types/mydecimal_serial_test.go b/types/mydecimal_serial_test.go index cae30ed8a04c6..cd72ef7ce131a 100644 --- a/types/mydecimal_serial_test.go +++ b/types/mydecimal_serial_test.go @@ -161,8 +161,8 @@ func TestFromStringMyDecimal(t *testing.T) { tests := []tcase{ {"12345", "12345", nil}, {"12345.", "12345", nil}, - {"123.45.", "123.45", nil}, - {"-123.45.", "-123.45", nil}, + {"123.45.", "123.45", ErrTruncated}, + {"-123.45.", "-123.45", ErrTruncated}, {".00012345000098765", "0.00012345000098765", nil}, {".12345000098765", "0.12345000098765", nil}, {"-.000000012345000098765", "-0.000000012345000098765", nil}, @@ -180,6 +180,10 @@ func TestFromStringMyDecimal(t *testing.T) { {"1e - 1", "1", ErrTruncated}, {"1e -1", "0.1", nil}, {"0.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", "0.000000000000000000000000000000000000000000000000000000000000000000000000", ErrTruncated}, + {"1asf", "1", ErrTruncated}, + {"1.1.1.1.1", "1.1", ErrTruncated}, + {"1 1", "1", ErrTruncated}, + {"1 ", "1", nil}, } dotest(tests) diff --git a/util/codec/codec.go b/util/codec/codec.go index f6daf7e54261d..a5657883a2648 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -449,7 +449,8 @@ func HashChunkSelected(sc *stmtctx.StatementContext, h []hash.Hash64, chk *chunk } case mysql.TypeDouble: f64s := column.Float64s() - for i, f := range f64s { + for i := range f64s { + f := f64s[i] if sel != nil && !sel[i] { continue } @@ -962,7 +963,7 @@ func peek(b []byte) (length int, err error) { return 0, errors.Trace(err) } length += l - if length < 0 { + if length <= 0 { return 0, errors.New("invalid encoded key") } else if length > originLength { return 0, errors.Errorf("invalid encoded key, "+ diff --git a/util/codec/codec_test.go b/util/codec/codec_test.go index ac41a148160cd..fef959a0c9c8b 100644 --- a/util/codec/codec_test.go +++ b/util/codec/codec_test.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/stretchr/testify/require" ) @@ -987,7 +988,7 @@ func TestDecodeOneToChunk(t *testing.T) { require.True(t, expect.IsNull()) } else { if got.Kind() != types.KindMysqlDecimal { - cmp, err := got.CompareDatum(sc, &expect) + cmp, err := got.Compare(sc, &expect, collate.GetCollator(tp.Collate)) require.NoError(t, err) require.Equalf(t, 0, cmp, "expect: %v, got %v", expect, got) } else { @@ -1110,7 +1111,7 @@ func TestDecodeRange(t *testing.T) { datums1, _, err := DecodeRange(rowData, len(datums), nil, nil) require.NoError(t, err) for i, datum := range datums1 { - cmp, err := datum.CompareDatum(nil, &datums[i]) + cmp, err := datum.Compare(nil, &datums[i], collate.GetBinaryCollator()) require.NoError(t, err) require.Equal(t, 0, cmp) } diff --git a/util/collate/collate.go b/util/collate/collate.go index 4f73b89b6742d..3cba9d6f572e5 100644 --- a/util/collate/collate.go +++ b/util/collate/collate.go @@ -154,6 +154,11 @@ func GetCollator(collate string) Collator { return binCollatorInstance } +// GetBinaryCollator gets the binary collator, it is often used when we want to apply binary compare. +func GetBinaryCollator() Collator { + return binCollatorInstance +} + // GetCollatorByID get the collator according to id, it will return the binary collator if the corresponding collator doesn't exist. func GetCollatorByID(id int) Collator { if atomic.LoadInt32(&newCollationEnabled) == 1 { diff --git a/util/hint/hint_processor.go b/util/hint/hint_processor.go index be0d5bcd02a2b..1bc49d92da301 100644 --- a/util/hint/hint_processor.go +++ b/util/hint/hint_processor.go @@ -299,7 +299,8 @@ func ParseHintsSet(p *parser.Parser, sql, charset, collation, db string) (*Hints func extractHintWarns(warns []error) []error { for _, w := range warns { - if parser.ErrWarnOptimizerHintUnsupportedHint.Equal(w) || + if parser.ErrParse.Equal(w) || + parser.ErrWarnOptimizerHintUnsupportedHint.Equal(w) || parser.ErrWarnOptimizerHintInvalidToken.Equal(w) || parser.ErrWarnMemoryQuotaOverflow.Equal(w) || parser.ErrWarnOptimizerHintParseError.Equal(w) || diff --git a/util/logutil/slow_query_logger.go b/util/logutil/slow_query_logger.go index 5f81f3d73b2f4..2588c36131fd9 100644 --- a/util/logutil/slow_query_logger.go +++ b/util/logutil/slow_query_logger.go @@ -29,14 +29,12 @@ var _pool = buffer.NewPool() func newSlowQueryLogger(cfg *LogConfig) (*zap.Logger, *log.ZapProperties, error) { - // copy global config and override slow query log file - // if slow query log filename is empty, slow query log will behave the same as global log + // copy the global log config to slow log config + // if the filename of slow log config is empty, slow log will behave the same as global log. sqConfig := cfg.Config if len(cfg.SlowQueryFile) != 0 { - sqConfig.File = log.FileLogConfig{ - MaxSize: cfg.File.MaxSize, - Filename: cfg.SlowQueryFile, - } + sqConfig.File = cfg.File + sqConfig.File.Filename = cfg.SlowQueryFile } // create the slow query logger diff --git a/expression/rand.go b/util/math/rand.go similarity index 74% rename from expression/rand.go rename to util/math/rand.go index f33331374f197..25b111e72788b 100644 --- a/expression/rand.go +++ b/util/math/rand.go @@ -12,9 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -package expression +package math -import "time" +import ( + "sync" + "time" +) const maxRandValue = 0x3FFFFFFF @@ -23,13 +26,18 @@ const maxRandValue = 0x3FFFFFFF type MysqlRng struct { seed1 uint32 seed2 uint32 + mu *sync.Mutex } // NewWithSeed create a rng with random seed. func NewWithSeed(seed int64) *MysqlRng { seed1 := uint32(seed*0x10001+55555555) % maxRandValue seed2 := uint32(seed*0x10000001) % maxRandValue - return &MysqlRng{seed1: seed1, seed2: seed2} + return &MysqlRng{ + seed1: seed1, + seed2: seed2, + mu: &sync.Mutex{}, + } } // NewWithTime create a rng with time stamp. @@ -39,7 +47,23 @@ func NewWithTime() *MysqlRng { // Gen will generate random number. func (rng *MysqlRng) Gen() float64 { + rng.mu.Lock() + defer rng.mu.Unlock() rng.seed1 = (rng.seed1*3 + rng.seed2) % maxRandValue rng.seed2 = (rng.seed1 + rng.seed2 + 33) % maxRandValue return float64(rng.seed1) / float64(maxRandValue) } + +// SetSeed1 is a interface to set seed1 +func (rng *MysqlRng) SetSeed1(seed uint32) { + rng.mu.Lock() + defer rng.mu.Unlock() + rng.seed1 = seed +} + +// SetSeed2 is a interface to set seed2 +func (rng *MysqlRng) SetSeed2(seed uint32) { + rng.mu.Lock() + defer rng.mu.Unlock() + rng.seed2 = seed +} diff --git a/expression/rand_test.go b/util/math/rand_test.go similarity index 80% rename from expression/rand_test.go rename to util/math/rand_test.go index 8a6f18b824718..70f6bac105e10 100644 --- a/expression/rand_test.go +++ b/util/math/rand_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package expression +package math import ( "testing" @@ -55,8 +55,23 @@ func TestRandWithSeed(t *testing.T) { for _, test := range tests { rng := NewWithSeed(test.seed) got1 := rng.Gen() - require.True(t, got1 == test.once) + require.Equal(t, got1, test.once) got2 := rng.Gen() - require.True(t, got2 == test.twice) + require.Equal(t, got2, test.twice) } } + +func TestRandWithSeed1AndSeed2(t *testing.T) { + t.Parallel() + + seed1 := uint32(10000000) + seed2 := uint32(1000000) + + rng := NewWithTime() + rng.SetSeed1(seed1) + rng.SetSeed2(seed2) + + require.Equal(t, rng.Gen(), 0.028870999839968048) + require.Equal(t, rng.Gen(), 0.11641535266900002) + require.Equal(t, rng.Gen(), 0.49546379455874096) +} diff --git a/util/memory/tracker.go b/util/memory/tracker.go index cda7a67ad278b..470029e309402 100644 --- a/util/memory/tracker.go +++ b/util/memory/tracker.go @@ -552,4 +552,8 @@ const ( LabelForSimpleTask int = -18 // LabelForCTEStorage represents the label of CTE storage LabelForCTEStorage int = -19 + // LabelForIndexJoinInnerWorker represents the label of IndexJoin InnerWorker + LabelForIndexJoinInnerWorker int = -20 + // LabelForIndexJoinOuterWorker represents the label of IndexJoin OuterWorker + LabelForIndexJoinOuterWorker int = -21 ) diff --git a/util/misc.go b/util/misc.go index f9256e180b716..21296da4086c9 100644 --- a/util/misc.go +++ b/util/misc.go @@ -166,6 +166,16 @@ func SyntaxWarn(err error) error { if err == nil { return nil } + logutil.BgLogger().Debug("syntax error", zap.Error(err)) + + // If the warn is already a terror with stack, pass it through. + if errors.HasStack(err) { + cause := errors.Cause(err) + if _, ok := cause.(*terror.Error); ok { + return err + } + } + return parser.ErrParse.GenWithStackByArgs(syntaxErrorPrefix, err.Error()) } diff --git a/util/pdapi/const.go b/util/pdapi/const.go index 5117b24809e8f..0ce75f5d5a6a0 100644 --- a/util/pdapi/const.go +++ b/util/pdapi/const.go @@ -18,6 +18,7 @@ package pdapi const ( HotRead = "/pd/api/v1/hotspot/regions/read" HotWrite = "/pd/api/v1/hotspot/regions/write" + HotHistory = "/pd/api/v1/hotspot/regions/history" Regions = "/pd/api/v1/regions" RegionByID = "/pd/api/v1/region/id/" Stores = "/pd/api/v1/stores" diff --git a/util/profile/flamegraph_test.go b/util/profile/flamegraph_test.go index 634e1286ef8cf..dfc83d3b27654 100644 --- a/util/profile/flamegraph_test.go +++ b/util/profile/flamegraph_test.go @@ -20,6 +20,7 @@ import ( "testing" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/collate" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -91,7 +92,7 @@ func TestProfileToDatum(t *testing.T) { comment = fmt.Sprintf("row %2d, actual (%s), expected (%s)", i, rowStr, expectStr) equal := true for j, r := range row { - v, err := r.CompareDatum(nil, &datums[i][j]) + v, err := r.Compare(nil, &datums[i][j], collate.GetBinaryCollator()) if v != 0 || err != nil { equal = false break diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index 9b03bb48f72db..f539f759c1675 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -216,7 +216,7 @@ func extractIndexPointRangesForCNF(sctx sessionctx.Context, conds []expression.E sameLens, allPoints := true, true numCols := int(0) for j, ran := range res.Ranges { - if !ran.IsPoint(sctx.GetSessionVars().StmtCtx) { + if !ran.IsPoint(sctx) { allPoints = false break } @@ -610,13 +610,12 @@ func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex // detachDNFCondAndBuildRangeForIndex will detach the index filters from table filters when it's a DNF. // We will detach the conditions of every DNF items, then compose them to a DNF. func (d *rangeDetacher) detachDNFCondAndBuildRangeForIndex(condition *expression.ScalarFunction, newTpSlice []*types.FieldType) ([]*Range, []expression.Expression, []*valueInfo, bool, error) { - sc := d.sctx.GetSessionVars().StmtCtx firstColumnChecker := &conditionChecker{ checkerCol: d.cols[0], shouldReserve: d.lengths[0] != types.UnspecifiedLength, length: d.lengths[0], } - rb := builder{sc: sc} + rb := builder{sc: d.sctx.GetSessionVars().StmtCtx} dnfItems := expression.FlattenDNFConditions(condition) newAccessItems := make([]expression.Expression, 0, len(dnfItems)) var totalRanges []*Range @@ -666,7 +665,7 @@ func (d *rangeDetacher) detachDNFCondAndBuildRangeForIndex(condition *expression firstColumnChecker.shouldReserve = d.lengths[0] != types.UnspecifiedLength } points := rb.build(item) - ranges, err := points2Ranges(sc, points, newTpSlice[0]) + ranges, err := points2Ranges(d.sctx, points, newTpSlice[0]) if err != nil { return nil, nil, nil, false, errors.Trace(err) } @@ -693,7 +692,7 @@ func (d *rangeDetacher) detachDNFCondAndBuildRangeForIndex(condition *expression if hasPrefix(d.lengths) { fixPrefixColRange(totalRanges, d.lengths, newTpSlice) } - totalRanges, err := UnionRanges(sc, totalRanges, d.mergeConsecutive) + totalRanges, err := UnionRanges(d.sctx, totalRanges, d.mergeConsecutive) if err != nil { return nil, nil, nil, false, errors.Trace(err) } diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index 7947696597e71..0e39a228ca9a3 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -28,12 +28,12 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" ) -func validInterval(sc *stmtctx.StatementContext, low, high *point) (bool, error) { +func validInterval(sctx sessionctx.Context, low, high *point) (bool, error) { + sc := sctx.GetSessionVars().StmtCtx l, err := codec.EncodeKey(sc, nil, low.value) if err != nil { return false, errors.Trace(err) @@ -53,18 +53,18 @@ func validInterval(sc *stmtctx.StatementContext, low, high *point) (bool, error) // points2Ranges build index ranges from range points. // Only one column is built there. If there're multiple columns, use appendPoints2Ranges. -func points2Ranges(sc *stmtctx.StatementContext, rangePoints []*point, tp *types.FieldType) ([]*Range, error) { +func points2Ranges(sctx sessionctx.Context, rangePoints []*point, tp *types.FieldType) ([]*Range, error) { ranges := make([]*Range, 0, len(rangePoints)/2) for i := 0; i < len(rangePoints); i += 2 { - startPoint, err := convertPoint(sc, rangePoints[i], tp) + startPoint, err := convertPoint(sctx, rangePoints[i], tp) if err != nil { return nil, errors.Trace(err) } - endPoint, err := convertPoint(sc, rangePoints[i+1], tp) + endPoint, err := convertPoint(sctx, rangePoints[i+1], tp) if err != nil { return nil, errors.Trace(err) } - less, err := validInterval(sc, startPoint, endPoint) + less, err := validInterval(sctx, startPoint, endPoint) if err != nil { return nil, errors.Trace(err) } @@ -87,7 +87,8 @@ func points2Ranges(sc *stmtctx.StatementContext, rangePoints []*point, tp *types return ranges, nil } -func convertPoint(sc *stmtctx.StatementContext, point *point, tp *types.FieldType) (*point, error) { +func convertPoint(sctx sessionctx.Context, point *point, tp *types.FieldType) (*point, error) { + sc := sctx.GetSessionVars().StmtCtx switch point.value.Kind() { case types.KindMaxValue, types.KindMinNotNull: return point, nil @@ -156,15 +157,15 @@ func convertPoint(sc *stmtctx.StatementContext, point *point, tp *types.FieldTyp // The additional column ranges can only be appended to point ranges. // for example we have an index (a, b), if the condition is (a > 1 and b = 2) // then we can not build a conjunctive ranges for this index. -func appendPoints2Ranges(sc *stmtctx.StatementContext, origin []*Range, rangePoints []*point, +func appendPoints2Ranges(sctx sessionctx.Context, origin []*Range, rangePoints []*point, ft *types.FieldType) ([]*Range, error) { var newIndexRanges []*Range for i := 0; i < len(origin); i++ { oRange := origin[i] - if !oRange.IsPoint(sc) { + if !oRange.IsPoint(sctx) { newIndexRanges = append(newIndexRanges, oRange) } else { - newRanges, err := appendPoints2IndexRange(sc, oRange, rangePoints, ft) + newRanges, err := appendPoints2IndexRange(sctx, oRange, rangePoints, ft) if err != nil { return nil, errors.Trace(err) } @@ -174,19 +175,19 @@ func appendPoints2Ranges(sc *stmtctx.StatementContext, origin []*Range, rangePoi return newIndexRanges, nil } -func appendPoints2IndexRange(sc *stmtctx.StatementContext, origin *Range, rangePoints []*point, +func appendPoints2IndexRange(sctx sessionctx.Context, origin *Range, rangePoints []*point, ft *types.FieldType) ([]*Range, error) { newRanges := make([]*Range, 0, len(rangePoints)/2) for i := 0; i < len(rangePoints); i += 2 { - startPoint, err := convertPoint(sc, rangePoints[i], ft) + startPoint, err := convertPoint(sctx, rangePoints[i], ft) if err != nil { return nil, errors.Trace(err) } - endPoint, err := convertPoint(sc, rangePoints[i+1], ft) + endPoint, err := convertPoint(sctx, rangePoints[i+1], ft) if err != nil { return nil, errors.Trace(err) } - less, err := validInterval(sc, startPoint, endPoint) + less, err := validInterval(sctx, startPoint, endPoint) if err != nil { return nil, errors.Trace(err) } @@ -236,7 +237,7 @@ func appendRanges2PointRanges(pointRanges []*Range, ranges []*Range) []*Range { // points2TableRanges build ranges for table scan from range points. // It will remove the nil and convert MinNotNull and MaxValue to MinInt64 or MinUint64 and MaxInt64 or MaxUint64. -func points2TableRanges(sc *stmtctx.StatementContext, rangePoints []*point, tp *types.FieldType) ([]*Range, error) { +func points2TableRanges(sctx sessionctx.Context, rangePoints []*point, tp *types.FieldType) ([]*Range, error) { ranges := make([]*Range, 0, len(rangePoints)/2) var minValueDatum, maxValueDatum types.Datum // Currently, table's kv range cannot accept encoded value of MaxValueDatum. we need to convert it. @@ -248,7 +249,7 @@ func points2TableRanges(sc *stmtctx.StatementContext, rangePoints []*point, tp * maxValueDatum.SetInt64(math.MaxInt64) } for i := 0; i < len(rangePoints); i += 2 { - startPoint, err := convertPoint(sc, rangePoints[i], tp) + startPoint, err := convertPoint(sctx, rangePoints[i], tp) if err != nil { return nil, errors.Trace(err) } @@ -258,7 +259,7 @@ func points2TableRanges(sc *stmtctx.StatementContext, rangePoints []*point, tp * } else if startPoint.value.Kind() == types.KindMinNotNull { startPoint.value = minValueDatum } - endPoint, err := convertPoint(sc, rangePoints[i+1], tp) + endPoint, err := convertPoint(sctx, rangePoints[i+1], tp) if err != nil { return nil, errors.Trace(err) } @@ -267,7 +268,7 @@ func points2TableRanges(sc *stmtctx.StatementContext, rangePoints []*point, tp * } else if endPoint.value.Kind() == types.KindNull { continue } - less, err := validInterval(sc, startPoint, endPoint) + less, err := validInterval(sctx, startPoint, endPoint) if err != nil { return nil, errors.Trace(err) } @@ -286,8 +287,8 @@ func points2TableRanges(sc *stmtctx.StatementContext, rangePoints []*point, tp * } // buildColumnRange builds range from CNF conditions. -func buildColumnRange(accessConditions []expression.Expression, sc *stmtctx.StatementContext, tp *types.FieldType, tableRange bool, colLen int) (ranges []*Range, err error) { - rb := builder{sc: sc} +func buildColumnRange(accessConditions []expression.Expression, sctx sessionctx.Context, tp *types.FieldType, tableRange bool, colLen int) (ranges []*Range, err error) { + rb := builder{sc: sctx.GetSessionVars().StmtCtx} rangePoints := getFullRange() for _, cond := range accessConditions { rangePoints = rb.intersection(rangePoints, rb.build(cond)) @@ -297,9 +298,9 @@ func buildColumnRange(accessConditions []expression.Expression, sc *stmtctx.Stat } newTp := newFieldType(tp) if tableRange { - ranges, err = points2TableRanges(sc, rangePoints, newTp) + ranges, err = points2TableRanges(sctx, rangePoints, newTp) } else { - ranges, err = points2Ranges(sc, rangePoints, newTp) + ranges, err = points2Ranges(sctx, rangePoints, newTp) } if err != nil { return nil, errors.Trace(err) @@ -316,7 +317,7 @@ func buildColumnRange(accessConditions []expression.Expression, sc *stmtctx.Stat ran.HighExclude = false } } - ranges, err = UnionRanges(sc, ranges, true) + ranges, err = UnionRanges(sctx, ranges, true) if err != nil { return nil, err } @@ -325,23 +326,22 @@ func buildColumnRange(accessConditions []expression.Expression, sc *stmtctx.Stat } // BuildTableRange builds range of PK column for PhysicalTableScan. -func BuildTableRange(accessConditions []expression.Expression, sc *stmtctx.StatementContext, tp *types.FieldType) ([]*Range, error) { - return buildColumnRange(accessConditions, sc, tp, true, types.UnspecifiedLength) +func BuildTableRange(accessConditions []expression.Expression, sctx sessionctx.Context, tp *types.FieldType) ([]*Range, error) { + return buildColumnRange(accessConditions, sctx, tp, true, types.UnspecifiedLength) } // BuildColumnRange builds range from access conditions for general columns. -func BuildColumnRange(conds []expression.Expression, sc *stmtctx.StatementContext, tp *types.FieldType, colLen int) ([]*Range, error) { +func BuildColumnRange(conds []expression.Expression, sctx sessionctx.Context, tp *types.FieldType, colLen int) ([]*Range, error) { if len(conds) == 0 { return []*Range{{LowVal: []types.Datum{{}}, HighVal: []types.Datum{types.MaxValueDatum()}}}, nil } - return buildColumnRange(conds, sc, tp, false, colLen) + return buildColumnRange(conds, sctx, tp, false, colLen) } // buildCNFIndexRange builds the range for index where the top layer is CNF. func (d *rangeDetacher) buildCNFIndexRange(newTp []*types.FieldType, eqAndInCount int, accessCondition []expression.Expression) ([]*Range, error) { - sc := d.sctx.GetSessionVars().StmtCtx - rb := builder{sc: sc} + rb := builder{sc: d.sctx.GetSessionVars().StmtCtx} var ( ranges []*Range err error @@ -356,9 +356,9 @@ func (d *rangeDetacher) buildCNFIndexRange(newTp []*types.FieldType, return nil, errors.Trace(rb.err) } if i == 0 { - ranges, err = points2Ranges(sc, point, newTp[i]) + ranges, err = points2Ranges(d.sctx, point, newTp[i]) } else { - ranges, err = appendPoints2Ranges(sc, ranges, point, newTp[i]) + ranges, err = appendPoints2Ranges(d.sctx, ranges, point, newTp[i]) } if err != nil { return nil, errors.Trace(err) @@ -373,9 +373,9 @@ func (d *rangeDetacher) buildCNFIndexRange(newTp []*types.FieldType, } } if eqAndInCount == 0 { - ranges, err = points2Ranges(sc, rangePoints, newTp[0]) + ranges, err = points2Ranges(d.sctx, rangePoints, newTp[0]) } else if eqAndInCount < len(accessCondition) { - ranges, err = appendPoints2Ranges(sc, ranges, rangePoints, newTp[eqAndInCount]) + ranges, err = appendPoints2Ranges(d.sctx, ranges, rangePoints, newTp[eqAndInCount]) } if err != nil { return nil, errors.Trace(err) @@ -384,7 +384,7 @@ func (d *rangeDetacher) buildCNFIndexRange(newTp []*types.FieldType, // Take prefix index into consideration. if hasPrefix(d.lengths) { if fixPrefixColRange(ranges, d.lengths, newTp) { - ranges, err = UnionRanges(sc, ranges, d.mergeConsecutive) + ranges, err = UnionRanges(d.sctx, ranges, d.mergeConsecutive) if err != nil { return nil, errors.Trace(err) } @@ -404,7 +404,8 @@ type sortRange struct { // For two intervals [a, b], [c, d], we have guaranteed that a <= c. If b >= c. Then two intervals are overlapped. // And this two can be merged as [a, max(b, d)]. // Otherwise they aren't overlapped. -func UnionRanges(sc *stmtctx.StatementContext, ranges []*Range, mergeConsecutive bool) ([]*Range, error) { +func UnionRanges(sctx sessionctx.Context, ranges []*Range, mergeConsecutive bool) ([]*Range, error) { + sc := sctx.GetSessionVars().StmtCtx if len(ranges) == 0 { return nil, nil } diff --git a/util/ranger/ranger_serial_test.go b/util/ranger/ranger_serial_test.go new file mode 100644 index 0000000000000..49cfc3638d060 --- /dev/null +++ b/util/ranger/ranger_serial_test.go @@ -0,0 +1,374 @@ +// 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 ranger_test + +import ( + "context" + "fmt" + "testing" + + "github.com/pingcap/tidb/expression" + plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/ranger" + "github.com/stretchr/testify/require" +) + +func TestIndexRange(t *testing.T) { + dom, store, err := newDomainStoreWithBootstrap(t) + defer func() { + dom.Close() + require.NoError(t, store.Close()) + }() + require.NoError(t, err) + testKit := testkit.NewTestKit(t, store) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec(` +create table t( + a varchar(50), + b int, + c double, + d varchar(10), + e binary(10), + f varchar(10) collate utf8mb4_general_ci, + g enum('A','B','C') collate utf8mb4_general_ci, + index idx_ab(a(50), b), + index idx_cb(c, a), + index idx_d(d(2)), + index idx_e(e(2)), + index idx_f(f), + index idx_de(d(2), e), + index idx_g(g) +)`) + + tests := []struct { + indexPos int + exprStr string + accessConds string + filterConds string + resultStr string + }{ + { + indexPos: 0, + exprStr: `a LIKE 'abc%'`, + accessConds: `[like(test.t.a, abc%, 92)]`, + filterConds: "[]", + resultStr: "[[\"abc\",\"abd\")]", + }, + { + indexPos: 0, + exprStr: "a LIKE 'abc_'", + accessConds: "[like(test.t.a, abc_, 92)]", + filterConds: "[like(test.t.a, abc_, 92)]", + resultStr: "[(\"abc\",\"abd\")]", + }, + { + indexPos: 0, + exprStr: "a LIKE 'abc'", + accessConds: "[like(test.t.a, abc, 92)]", + filterConds: "[]", + resultStr: "[[\"abc\",\"abc\"]]", + }, + { + indexPos: 0, + exprStr: `a LIKE "ab\_c"`, + accessConds: "[like(test.t.a, ab\\_c, 92)]", + filterConds: "[]", + resultStr: "[[\"ab_c\",\"ab_c\"]]", + }, + { + indexPos: 0, + exprStr: `a LIKE '%'`, + accessConds: "[]", + filterConds: `[like(test.t.a, %, 92)]`, + resultStr: "[[NULL,+inf]]", + }, + { + indexPos: 0, + exprStr: `a LIKE '\%a'`, + accessConds: "[like(test.t.a, \\%a, 92)]", + filterConds: "[]", + resultStr: `[["%a","%a"]]`, + }, + { + indexPos: 0, + exprStr: `a LIKE "\\"`, + accessConds: "[like(test.t.a, \\, 92)]", + filterConds: "[]", + resultStr: "[[\"\\\",\"\\\"]]", + }, + { + indexPos: 0, + exprStr: `a LIKE "\\\\a%"`, + accessConds: `[like(test.t.a, \\a%, 92)]`, + filterConds: "[]", + resultStr: "[[\"\\a\",\"\\b\")]", + }, + { + indexPos: 0, + exprStr: `a > NULL`, + accessConds: "[gt(test.t.a, )]", + filterConds: "[]", + resultStr: `[]`, + }, + { + indexPos: 0, + exprStr: `a = 'a' and b in (1, 2, 3)`, + accessConds: "[eq(test.t.a, a) in(test.t.b, 1, 2, 3)]", + filterConds: "[]", + resultStr: "[[\"a\" 1,\"a\" 1] [\"a\" 2,\"a\" 2] [\"a\" 3,\"a\" 3]]", + }, + { + indexPos: 0, + exprStr: `a = 'a' and b not in (1, 2, 3)`, + accessConds: "[eq(test.t.a, a) not(in(test.t.b, 1, 2, 3))]", + filterConds: "[]", + resultStr: "[(\"a\" NULL,\"a\" 1) (\"a\" 3,\"a\" +inf]]", + }, + { + indexPos: 0, + exprStr: `a in ('a') and b in ('1', 2.0, NULL)`, + accessConds: "[eq(test.t.a, a) in(test.t.b, 1, 2, )]", + filterConds: "[]", + resultStr: `[["a" 1,"a" 1] ["a" 2,"a" 2]]`, + }, + { + indexPos: 1, + exprStr: `c in ('1.1', 1, 1.1) and a in ('1', 'a', NULL)`, + accessConds: "[in(test.t.c, 1.1, 1, 1.1) in(test.t.a, 1, a, )]", + filterConds: "[]", + resultStr: "[[1 \"1\",1 \"1\"] [1 \"a\",1 \"a\"] [1.1 \"1\",1.1 \"1\"] [1.1 \"a\",1.1 \"a\"]]", + }, + { + indexPos: 1, + exprStr: "c in (1, 1, 1, 1, 1, 1, 2, 1, 2, 3, 2, 3, 4, 4, 1, 2)", + accessConds: "[in(test.t.c, 1, 1, 1, 1, 1, 1, 2, 1, 2, 3, 2, 3, 4, 4, 1, 2)]", + filterConds: "[]", + resultStr: "[[1,1] [2,2] [3,3] [4,4]]", + }, + { + indexPos: 1, + exprStr: "c not in (1, 2, 3)", + accessConds: "[not(in(test.t.c, 1, 2, 3))]", + filterConds: "[]", + resultStr: "[(NULL,1) (1,2) (2,3) (3,+inf]]", + }, + { + indexPos: 1, + exprStr: "c in (1, 2) and c in (1, 3)", + accessConds: "[eq(test.t.c, 1)]", + filterConds: "[]", + resultStr: "[[1,1]]", + }, + { + indexPos: 1, + exprStr: "c = 1 and c = 2", + accessConds: "[]", + filterConds: "[]", + resultStr: "[]", + }, + { + indexPos: 0, + exprStr: "a in (NULL)", + accessConds: "[eq(test.t.a, )]", + filterConds: "[]", + resultStr: "[]", + }, + { + indexPos: 0, + exprStr: "a not in (NULL, '1', '2', '3')", + accessConds: "[not(in(test.t.a, , 1, 2, 3))]", + filterConds: "[]", + resultStr: "[]", + }, + { + indexPos: 0, + exprStr: "not (a not in (NULL, '1', '2', '3') and a > '2')", + accessConds: "[or(in(test.t.a, , 1, 2, 3), le(test.t.a, 2))]", + filterConds: "[]", + resultStr: "[[-inf,\"2\"] [\"3\",\"3\"]]", + }, + { + indexPos: 0, + exprStr: "not (a not in (NULL) and a > '2')", + accessConds: "[or(eq(test.t.a, ), le(test.t.a, 2))]", + filterConds: "[]", + resultStr: "[[-inf,\"2\"]]", + }, + { + indexPos: 0, + exprStr: "not (a not in (NULL) or a > '2')", + accessConds: "[and(eq(test.t.a, ), le(test.t.a, 2))]", + filterConds: "[]", + resultStr: "[]", + }, + { + indexPos: 0, + exprStr: "(a > 'b' and a < 'bbb') or (a < 'cb' and a > 'a')", + accessConds: "[or(and(gt(test.t.a, b), lt(test.t.a, bbb)), and(lt(test.t.a, cb), gt(test.t.a, a)))]", + filterConds: "[]", + resultStr: "[(\"a\",\"cb\")]", + }, + { + indexPos: 0, + exprStr: "(a > 'a' and a < 'b') or (a >= 'b' and a < 'c')", + accessConds: "[or(and(gt(test.t.a, a), lt(test.t.a, b)), and(ge(test.t.a, b), lt(test.t.a, c)))]", + filterConds: "[]", + resultStr: "[(\"a\",\"c\")]", + }, + { + indexPos: 0, + exprStr: "(a > 'a' and a < 'b' and b < 1) or (a >= 'b' and a < 'c')", + accessConds: "[or(and(gt(test.t.a, a), lt(test.t.a, b)), and(ge(test.t.a, b), lt(test.t.a, c)))]", + filterConds: "[or(and(and(gt(test.t.a, a), lt(test.t.a, b)), lt(test.t.b, 1)), and(ge(test.t.a, b), lt(test.t.a, c)))]", + resultStr: "[(\"a\",\"c\")]", + }, + { + indexPos: 0, + exprStr: "(a in ('a', 'b') and b < 1) or (a >= 'b' and a < 'c')", + accessConds: "[or(and(in(test.t.a, a, b), lt(test.t.b, 1)), and(ge(test.t.a, b), lt(test.t.a, c)))]", + filterConds: "[]", + resultStr: `[["a" -inf,"a" 1) ["b","c")]`, + }, + { + indexPos: 0, + exprStr: "(a > 'a') or (c > 1)", + accessConds: "[]", + filterConds: "[or(gt(test.t.a, a), gt(test.t.c, 1))]", + resultStr: "[[NULL,+inf]]", + }, + { + indexPos: 2, + exprStr: `d = "你好啊"`, + accessConds: "[eq(test.t.d, 你好啊)]", + filterConds: "[eq(test.t.d, 你好啊)]", + resultStr: "[[\"你好\",\"你好\"]]", + }, + { + indexPos: 3, + exprStr: `e = "你好啊"`, + accessConds: "[eq(test.t.e, 你好啊)]", + filterConds: "[eq(test.t.e, 你好啊)]", + resultStr: "[[0xE4BD,0xE4BD]]", + }, + { + indexPos: 2, + exprStr: `d in ("你好啊", "再见")`, + accessConds: "[in(test.t.d, 你好啊, 再见)]", + filterConds: "[in(test.t.d, 你好啊, 再见)]", + resultStr: "[[\"你好\",\"你好\"] [\"再见\",\"再见\"]]", + }, + { + indexPos: 2, + exprStr: `d not in ("你好啊")`, + accessConds: "[]", + filterConds: "[ne(test.t.d, 你好啊)]", + resultStr: "[[NULL,+inf]]", + }, + { + indexPos: 2, + exprStr: `d < "你好" || d > "你好"`, + accessConds: "[or(lt(test.t.d, 你好), gt(test.t.d, 你好))]", + filterConds: "[or(lt(test.t.d, 你好), gt(test.t.d, 你好))]", + resultStr: "[[-inf,+inf]]", + }, + { + indexPos: 2, + exprStr: `not(d < "你好" || d > "你好")`, + accessConds: "[and(ge(test.t.d, 你好), le(test.t.d, 你好))]", + filterConds: "[and(ge(test.t.d, 你好), le(test.t.d, 你好))]", + resultStr: "[[\"你好\",\"你好\"]]", + }, + { + indexPos: 4, + exprStr: "f >= 'a' and f <= 'B'", + accessConds: "[ge(test.t.f, a) le(test.t.f, B)]", + filterConds: "[]", + resultStr: "[[\"a\",\"B\"]]", + }, + { + indexPos: 4, + exprStr: "f in ('a', 'B')", + accessConds: "[in(test.t.f, a, B)]", + filterConds: "[]", + resultStr: "[[\"a\",\"a\"] [\"B\",\"B\"]]", + }, + { + indexPos: 4, + exprStr: "f = 'a' and f = 'B' collate utf8mb4_bin", + accessConds: "[eq(test.t.f, a)]", + filterConds: "[eq(test.t.f, B)]", + resultStr: "[[\"a\",\"a\"]]", + }, + { + indexPos: 4, + exprStr: "f like '@%' collate utf8mb4_bin", + accessConds: "[]", + filterConds: "[like(test.t.f, @%, 92)]", + resultStr: "[[NULL,+inf]]", + }, + { + indexPos: 5, + exprStr: "d in ('aab', 'aac') and e = 'a'", + accessConds: "[in(test.t.d, aab, aac) eq(test.t.e, a)]", + filterConds: "[in(test.t.d, aab, aac)]", + resultStr: "[[\"aa\" 0x61,\"aa\" 0x61]]", + }, + { + indexPos: 6, + exprStr: "g = 'a'", + accessConds: "[eq(test.t.g, a)]", + filterConds: "[]", + resultStr: "[[\"A\",\"A\"]]", + }, + } + + collate.SetNewCollationEnabledForTest(true) + defer func() { collate.SetNewCollationEnabledForTest(false) }() + ctx := context.Background() + for _, tt := range tests { + t.Run(tt.exprStr, func(t *testing.T) { + sql := "select * from t where " + tt.exprStr + sctx := testKit.Session().(sessionctx.Context) + stmts, err := session.Parse(sctx, sql) + require.NoError(t, err) + require.Len(t, stmts, 1) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) + require.NoError(t, err) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) + require.NoError(t, err) + selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo() + require.NotNil(t, selection) + conds := make([]expression.Expression, len(selection.Conditions)) + for i, cond := range selection.Conditions { + conds[i] = expression.PushDownNot(sctx, cond) + } + cols, lengths := expression.IndexInfo2PrefixCols(tbl.Columns, selection.Schema().Columns, tbl.Indices[tt.indexPos]) + require.NotNil(t, cols) + res, err := ranger.DetachCondAndBuildRangeForIndex(sctx, conds, cols, lengths) + require.NoError(t, err) + require.Equal(t, tt.accessConds, fmt.Sprintf("%s", res.AccessConds)) + require.Equal(t, tt.filterConds, fmt.Sprintf("%s", res.RemainedConds)) + got := fmt.Sprintf("%v", res.Ranges) + require.Equal(t, tt.resultStr, got) + }) + } +} diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index cd2d42572e5f5..f0aaa9b09ee7b 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -17,7 +17,6 @@ package ranger_test import ( "context" "fmt" - "testing" "github.com/pingcap/errors" @@ -27,13 +26,11 @@ import ( 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/store/mockstore" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/ranger" "github.com/stretchr/testify/require" ) @@ -300,7 +297,7 @@ func TestTableRange(t *testing.T) { conds, filter = ranger.DetachCondsForColumn(sctx, conds, col) require.Equal(t, tt.accessConds, fmt.Sprintf("%s", conds)) require.Equal(t, tt.filterConds, fmt.Sprintf("%s", filter)) - result, err := ranger.BuildTableRange(conds, new(stmtctx.StatementContext), col.RetType) + result, err := ranger.BuildTableRange(conds, sctx, col.RetType) require.NoError(t, err) got := fmt.Sprintf("%v", result) require.Equal(t, tt.resultStr, got) @@ -308,351 +305,6 @@ func TestTableRange(t *testing.T) { } } -func TestIndexRange(t *testing.T) { - t.Parallel() - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) - testKit := testkit.NewTestKit(t, store) - testKit.MustExec("use test") - testKit.MustExec("drop table if exists t") - testKit.MustExec(` -create table t( - a varchar(50), - b int, - c double, - d varchar(10), - e binary(10), - f varchar(10) collate utf8mb4_general_ci, - g enum('A','B','C') collate utf8mb4_general_ci, - index idx_ab(a(50), b), - index idx_cb(c, a), - index idx_d(d(2)), - index idx_e(e(2)), - index idx_f(f), - index idx_de(d(2), e), - index idx_g(g) -)`) - - tests := []struct { - indexPos int - exprStr string - accessConds string - filterConds string - resultStr string - }{ - { - indexPos: 0, - exprStr: `a LIKE 'abc%'`, - accessConds: `[like(test.t.a, abc%, 92)]`, - filterConds: "[]", - resultStr: "[[\"abc\",\"abd\")]", - }, - { - indexPos: 0, - exprStr: "a LIKE 'abc_'", - accessConds: "[like(test.t.a, abc_, 92)]", - filterConds: "[like(test.t.a, abc_, 92)]", - resultStr: "[(\"abc\",\"abd\")]", - }, - { - indexPos: 0, - exprStr: "a LIKE 'abc'", - accessConds: "[like(test.t.a, abc, 92)]", - filterConds: "[]", - resultStr: "[[\"abc\",\"abc\"]]", - }, - { - indexPos: 0, - exprStr: `a LIKE "ab\_c"`, - accessConds: "[like(test.t.a, ab\\_c, 92)]", - filterConds: "[]", - resultStr: "[[\"ab_c\",\"ab_c\"]]", - }, - { - indexPos: 0, - exprStr: `a LIKE '%'`, - accessConds: "[]", - filterConds: `[like(test.t.a, %, 92)]`, - resultStr: "[[NULL,+inf]]", - }, - { - indexPos: 0, - exprStr: `a LIKE '\%a'`, - accessConds: "[like(test.t.a, \\%a, 92)]", - filterConds: "[]", - resultStr: `[["%a","%a"]]`, - }, - { - indexPos: 0, - exprStr: `a LIKE "\\"`, - accessConds: "[like(test.t.a, \\, 92)]", - filterConds: "[]", - resultStr: "[[\"\\\",\"\\\"]]", - }, - { - indexPos: 0, - exprStr: `a LIKE "\\\\a%"`, - accessConds: `[like(test.t.a, \\a%, 92)]`, - filterConds: "[]", - resultStr: "[[\"\\a\",\"\\b\")]", - }, - { - indexPos: 0, - exprStr: `a > NULL`, - accessConds: "[gt(test.t.a, )]", - filterConds: "[]", - resultStr: `[]`, - }, - { - indexPos: 0, - exprStr: `a = 'a' and b in (1, 2, 3)`, - accessConds: "[eq(test.t.a, a) in(test.t.b, 1, 2, 3)]", - filterConds: "[]", - resultStr: "[[\"a\" 1,\"a\" 1] [\"a\" 2,\"a\" 2] [\"a\" 3,\"a\" 3]]", - }, - { - indexPos: 0, - exprStr: `a = 'a' and b not in (1, 2, 3)`, - accessConds: "[eq(test.t.a, a) not(in(test.t.b, 1, 2, 3))]", - filterConds: "[]", - resultStr: "[(\"a\" NULL,\"a\" 1) (\"a\" 3,\"a\" +inf]]", - }, - { - indexPos: 0, - exprStr: `a in ('a') and b in ('1', 2.0, NULL)`, - accessConds: "[eq(test.t.a, a) in(test.t.b, 1, 2, )]", - filterConds: "[]", - resultStr: `[["a" 1,"a" 1] ["a" 2,"a" 2]]`, - }, - { - indexPos: 1, - exprStr: `c in ('1.1', 1, 1.1) and a in ('1', 'a', NULL)`, - accessConds: "[in(test.t.c, 1.1, 1, 1.1) in(test.t.a, 1, a, )]", - filterConds: "[]", - resultStr: "[[1 \"1\",1 \"1\"] [1 \"a\",1 \"a\"] [1.1 \"1\",1.1 \"1\"] [1.1 \"a\",1.1 \"a\"]]", - }, - { - indexPos: 1, - exprStr: "c in (1, 1, 1, 1, 1, 1, 2, 1, 2, 3, 2, 3, 4, 4, 1, 2)", - accessConds: "[in(test.t.c, 1, 1, 1, 1, 1, 1, 2, 1, 2, 3, 2, 3, 4, 4, 1, 2)]", - filterConds: "[]", - resultStr: "[[1,1] [2,2] [3,3] [4,4]]", - }, - { - indexPos: 1, - exprStr: "c not in (1, 2, 3)", - accessConds: "[not(in(test.t.c, 1, 2, 3))]", - filterConds: "[]", - resultStr: "[(NULL,1) (1,2) (2,3) (3,+inf]]", - }, - { - indexPos: 1, - exprStr: "c in (1, 2) and c in (1, 3)", - accessConds: "[eq(test.t.c, 1)]", - filterConds: "[]", - resultStr: "[[1,1]]", - }, - { - indexPos: 1, - exprStr: "c = 1 and c = 2", - accessConds: "[]", - filterConds: "[]", - resultStr: "[]", - }, - { - indexPos: 0, - exprStr: "a in (NULL)", - accessConds: "[eq(test.t.a, )]", - filterConds: "[]", - resultStr: "[]", - }, - { - indexPos: 0, - exprStr: "a not in (NULL, '1', '2', '3')", - accessConds: "[not(in(test.t.a, , 1, 2, 3))]", - filterConds: "[]", - resultStr: "[]", - }, - { - indexPos: 0, - exprStr: "not (a not in (NULL, '1', '2', '3') and a > '2')", - accessConds: "[or(in(test.t.a, , 1, 2, 3), le(test.t.a, 2))]", - filterConds: "[]", - resultStr: "[[-inf,\"2\"] [\"3\",\"3\"]]", - }, - { - indexPos: 0, - exprStr: "not (a not in (NULL) and a > '2')", - accessConds: "[or(eq(test.t.a, ), le(test.t.a, 2))]", - filterConds: "[]", - resultStr: "[[-inf,\"2\"]]", - }, - { - indexPos: 0, - exprStr: "not (a not in (NULL) or a > '2')", - accessConds: "[and(eq(test.t.a, ), le(test.t.a, 2))]", - filterConds: "[]", - resultStr: "[]", - }, - { - indexPos: 0, - exprStr: "(a > 'b' and a < 'bbb') or (a < 'cb' and a > 'a')", - accessConds: "[or(and(gt(test.t.a, b), lt(test.t.a, bbb)), and(lt(test.t.a, cb), gt(test.t.a, a)))]", - filterConds: "[]", - resultStr: "[(\"a\",\"cb\")]", - }, - { - indexPos: 0, - exprStr: "(a > 'a' and a < 'b') or (a >= 'b' and a < 'c')", - accessConds: "[or(and(gt(test.t.a, a), lt(test.t.a, b)), and(ge(test.t.a, b), lt(test.t.a, c)))]", - filterConds: "[]", - resultStr: "[(\"a\",\"c\")]", - }, - { - indexPos: 0, - exprStr: "(a > 'a' and a < 'b' and b < 1) or (a >= 'b' and a < 'c')", - accessConds: "[or(and(gt(test.t.a, a), lt(test.t.a, b)), and(ge(test.t.a, b), lt(test.t.a, c)))]", - filterConds: "[or(and(and(gt(test.t.a, a), lt(test.t.a, b)), lt(test.t.b, 1)), and(ge(test.t.a, b), lt(test.t.a, c)))]", - resultStr: "[(\"a\",\"c\")]", - }, - { - indexPos: 0, - exprStr: "(a in ('a', 'b') and b < 1) or (a >= 'b' and a < 'c')", - accessConds: "[or(and(in(test.t.a, a, b), lt(test.t.b, 1)), and(ge(test.t.a, b), lt(test.t.a, c)))]", - filterConds: "[]", - resultStr: `[["a" -inf,"a" 1) ["b","c")]`, - }, - { - indexPos: 0, - exprStr: "(a > 'a') or (c > 1)", - accessConds: "[]", - filterConds: "[or(gt(test.t.a, a), gt(test.t.c, 1))]", - resultStr: "[[NULL,+inf]]", - }, - { - indexPos: 2, - exprStr: `d = "你好啊"`, - accessConds: "[eq(test.t.d, 你好啊)]", - filterConds: "[eq(test.t.d, 你好啊)]", - resultStr: "[[\"你好\",\"你好\"]]", - }, - { - indexPos: 3, - exprStr: `e = "你好啊"`, - accessConds: "[eq(test.t.e, 你好啊)]", - filterConds: "[eq(test.t.e, 你好啊)]", - resultStr: "[[0xE4BD,0xE4BD]]", - }, - { - indexPos: 2, - exprStr: `d in ("你好啊", "再见")`, - accessConds: "[in(test.t.d, 你好啊, 再见)]", - filterConds: "[in(test.t.d, 你好啊, 再见)]", - resultStr: "[[\"你好\",\"你好\"] [\"再见\",\"再见\"]]", - }, - { - indexPos: 2, - exprStr: `d not in ("你好啊")`, - accessConds: "[]", - filterConds: "[ne(test.t.d, 你好啊)]", - resultStr: "[[NULL,+inf]]", - }, - { - indexPos: 2, - exprStr: `d < "你好" || d > "你好"`, - accessConds: "[or(lt(test.t.d, 你好), gt(test.t.d, 你好))]", - filterConds: "[or(lt(test.t.d, 你好), gt(test.t.d, 你好))]", - resultStr: "[[-inf,+inf]]", - }, - { - indexPos: 2, - exprStr: `not(d < "你好" || d > "你好")`, - accessConds: "[and(ge(test.t.d, 你好), le(test.t.d, 你好))]", - filterConds: "[and(ge(test.t.d, 你好), le(test.t.d, 你好))]", - resultStr: "[[\"你好\",\"你好\"]]", - }, - { - indexPos: 4, - exprStr: "f >= 'a' and f <= 'B'", - accessConds: "[ge(test.t.f, a) le(test.t.f, B)]", - filterConds: "[]", - resultStr: "[[\"a\",\"B\"]]", - }, - { - indexPos: 4, - exprStr: "f in ('a', 'B')", - accessConds: "[in(test.t.f, a, B)]", - filterConds: "[]", - resultStr: "[[\"a\",\"a\"] [\"B\",\"B\"]]", - }, - { - indexPos: 4, - exprStr: "f = 'a' and f = 'B' collate utf8mb4_bin", - accessConds: "[eq(test.t.f, a)]", - filterConds: "[eq(test.t.f, B)]", - resultStr: "[[\"a\",\"a\"]]", - }, - { - indexPos: 4, - exprStr: "f like '@%' collate utf8mb4_bin", - accessConds: "[]", - filterConds: "[like(test.t.f, @%, 92)]", - resultStr: "[[NULL,+inf]]", - }, - { - indexPos: 5, - exprStr: "d in ('aab', 'aac') and e = 'a'", - accessConds: "[in(test.t.d, aab, aac) eq(test.t.e, a)]", - filterConds: "[in(test.t.d, aab, aac)]", - resultStr: "[[\"aa\" 0x61,\"aa\" 0x61]]", - }, - { - indexPos: 6, - exprStr: "g = 'a'", - accessConds: "[eq(test.t.g, a)]", - filterConds: "[]", - resultStr: "[[\"A\",\"A\"]]", - }, - } - - collate.SetNewCollationEnabledForTest(true) - defer func() { collate.SetNewCollationEnabledForTest(false) }() - ctx := context.Background() - for _, tt := range tests { - t.Run(tt.exprStr, func(t *testing.T) { - sql := "select * from t where " + tt.exprStr - sctx := testKit.Session().(sessionctx.Context) - stmts, err := session.Parse(sctx, sql) - require.NoError(t, err) - require.Len(t, stmts, 1) - ret := &plannercore.PreprocessorReturn{} - err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) - require.NoError(t, err) - p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) - require.NoError(t, err) - selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) - tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo() - require.NotNil(t, selection) - conds := make([]expression.Expression, len(selection.Conditions)) - for i, cond := range selection.Conditions { - conds[i] = expression.PushDownNot(sctx, cond) - } - cols, lengths := expression.IndexInfo2PrefixCols(tbl.Columns, selection.Schema().Columns, tbl.Indices[tt.indexPos]) - require.NotNil(t, cols) - res, err := ranger.DetachCondAndBuildRangeForIndex(sctx, conds, cols, lengths) - require.NoError(t, err) - require.Equal(t, tt.accessConds, fmt.Sprintf("%s", res.AccessConds)) - require.Equal(t, tt.filterConds, fmt.Sprintf("%s", res.RemainedConds)) - got := fmt.Sprintf("%v", res.Ranges) - require.Equal(t, tt.resultStr, got) - }) - } -} - // for issue #6661 func TestIndexRangeForUnsignedAndOverflow(t *testing.T) { t.Parallel() @@ -1203,7 +855,7 @@ func TestColumnRange(t *testing.T) { require.NotNil(t, col) conds = ranger.ExtractAccessConditionsForColumn(conds, col) require.Equal(t, tt.accessConds, fmt.Sprintf("%s", conds)) - result, err := ranger.BuildColumnRange(conds, new(stmtctx.StatementContext), col.RetType, tt.length) + result, err := ranger.BuildColumnRange(conds, sctx, col.RetType, tt.length) require.NoError(t, err) got := fmt.Sprintf("%v", result) require.Equal(t, tt.resultStr, got) diff --git a/util/ranger/types.go b/util/ranger/types.go index 8559a404d80d5..8adb8f9389485 100644 --- a/util/ranger/types.go +++ b/util/ranger/types.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" @@ -80,33 +81,11 @@ func (ran *Range) Clone() *Range { } // IsPoint returns if the range is a point. -func (ran *Range) IsPoint(sc *stmtctx.StatementContext) bool { - if len(ran.LowVal) != len(ran.HighVal) { - return false - } - for i := range ran.LowVal { - a := ran.LowVal[i] - b := ran.HighVal[i] - if a.Kind() == types.KindMinNotNull || b.Kind() == types.KindMaxValue { - return false - } - cmp, err := a.CompareDatum(sc, &b) - if err != nil { - return false - } - if cmp != 0 { - return false - } - - if a.IsNull() { - return false - } - } - return !ran.LowExclude && !ran.HighExclude +func (ran *Range) IsPoint(sctx sessionctx.Context) bool { + return ran.isPoint(sctx, false) } -// IsPointNullable returns if the range is a point. -func (ran *Range) IsPointNullable(sc *stmtctx.StatementContext) bool { +func (ran *Range) isPoint(sctx sessionctx.Context, regardNullAsPoint bool) bool { if len(ran.LowVal) != len(ran.HighVal) { return false } @@ -116,7 +95,7 @@ func (ran *Range) IsPointNullable(sc *stmtctx.StatementContext) bool { if a.Kind() == types.KindMinNotNull || b.Kind() == types.KindMaxValue { return false } - cmp, err := a.CompareDatum(sc, &b) + cmp, err := a.CompareDatum(sctx.GetSessionVars().StmtCtx, &b) if err != nil { return false } @@ -124,8 +103,8 @@ func (ran *Range) IsPointNullable(sc *stmtctx.StatementContext) bool { return false } - if a.IsNull() { - if !b.IsNull() { + if a.IsNull() && b.IsNull() { // [NULL, NULL] + if !regardNullAsPoint { return false } } @@ -133,6 +112,11 @@ func (ran *Range) IsPointNullable(sc *stmtctx.StatementContext) bool { return !ran.LowExclude && !ran.HighExclude } +// IsPointNullable returns if the range is a point. +func (ran *Range) IsPointNullable(sctx sessionctx.Context) bool { + return ran.isPoint(sctx, true) +} + // IsFullRange check if the range is full scan range func (ran *Range) IsFullRange(unsignedIntHandle bool) bool { if unsignedIntHandle { diff --git a/util/ranger/types_test.go b/util/ranger/types_test.go index 19877cae9b5cd..1870f8da01ff6 100644 --- a/util/ranger/types_test.go +++ b/util/ranger/types_test.go @@ -18,7 +18,7 @@ import ( "math" "testing" - "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" "github.com/stretchr/testify/require" @@ -124,9 +124,8 @@ func TestRange(t *testing.T) { isPoint: false, }, } - sc := new(stmtctx.StatementContext) for _, v := range isPointTests { - require.Equal(t, v.isPoint, v.ran.IsPoint(sc)) + require.Equal(t, v.isPoint, v.ran.IsPoint(core.MockContext())) } } diff --git a/util/resourcegrouptag/resource_group_tag.go b/util/resourcegrouptag/resource_group_tag.go index bf63a08cd6d07..1b70c6e207ea4 100644 --- a/util/resourcegrouptag/resource_group_tag.go +++ b/util/resourcegrouptag/resource_group_tag.go @@ -16,17 +16,21 @@ package resourcegrouptag import ( "github.com/pingcap/errors" + "github.com/pingcap/kvproto/pkg/coprocessor" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/tablecodec/rowindexcodec" "github.com/pingcap/tipb/go-tipb" + "github.com/tikv/client-go/v2/tikvrpc" ) // EncodeResourceGroupTag encodes sql digest and plan digest into resource group tag. -func EncodeResourceGroupTag(sqlDigest, planDigest *parser.Digest) []byte { +func EncodeResourceGroupTag(sqlDigest, planDigest *parser.Digest, label tipb.ResourceGroupTagLabel) []byte { if sqlDigest == nil && planDigest == nil { return nil } - tag := &tipb.ResourceGroupTag{} + tag := &tipb.ResourceGroupTag{Label: &label} if sqlDigest != nil { tag.SqlDigest = sqlDigest.Bytes() } @@ -52,3 +56,75 @@ func DecodeResourceGroupTag(data []byte) (sqlDigest []byte, err error) { } return tag.SqlDigest, nil } + +// GetResourceGroupLabelByKey determines the tipb.ResourceGroupTagLabel of key. +func GetResourceGroupLabelByKey(key []byte) tipb.ResourceGroupTagLabel { + switch rowindexcodec.GetKeyKind(key) { + case rowindexcodec.KeyKindRow: + return tipb.ResourceGroupTagLabel_ResourceGroupTagLabelRow + case rowindexcodec.KeyKindIndex: + return tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex + default: + return tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown + } +} + +// GetFirstKeyFromRequest gets the first Key of the request from tikvrpc.Request. +func GetFirstKeyFromRequest(req *tikvrpc.Request) (firstKey []byte) { + if req == nil { + return + } + switch req.Req.(type) { + case *kvrpcpb.GetRequest: + r := req.Req.(*kvrpcpb.GetRequest) + if r != nil { + firstKey = r.Key + } + case *kvrpcpb.BatchGetRequest: + r := req.Req.(*kvrpcpb.BatchGetRequest) + if r != nil && len(r.Keys) > 0 { + firstKey = r.Keys[0] + } + case *kvrpcpb.ScanRequest: + r := req.Req.(*kvrpcpb.ScanRequest) + if r != nil { + firstKey = r.StartKey + } + case *kvrpcpb.PrewriteRequest: + r := req.Req.(*kvrpcpb.PrewriteRequest) + if r != nil && len(r.Mutations) > 0 { + if mutation := r.Mutations[0]; mutation != nil { + firstKey = mutation.Key + } + } + case *kvrpcpb.CommitRequest: + r := req.Req.(*kvrpcpb.CommitRequest) + if r != nil && len(r.Keys) > 0 { + firstKey = r.Keys[0] + } + case *kvrpcpb.BatchRollbackRequest: + r := req.Req.(*kvrpcpb.BatchRollbackRequest) + if r != nil && len(r.Keys) > 0 { + firstKey = r.Keys[0] + } + case *coprocessor.Request: + r := req.Req.(*coprocessor.Request) + if r != nil && len(r.Ranges) > 0 { + if keyRange := r.Ranges[0]; keyRange != nil { + firstKey = keyRange.Start + } + } + case *coprocessor.BatchRequest: + r := req.Req.(*coprocessor.BatchRequest) + if r != nil && len(r.Regions) > 0 { + if region := r.Regions[0]; region != nil { + if len(region.Ranges) > 0 { + if keyRange := region.Ranges[0]; keyRange != nil { + firstKey = keyRange.Start + } + } + } + } + } + return +} diff --git a/util/resourcegrouptag/resource_group_tag_test.go b/util/resourcegrouptag/resource_group_tag_test.go index 355134251a020..5cf65d52ad411 100644 --- a/util/resourcegrouptag/resource_group_tag_test.go +++ b/util/resourcegrouptag/resource_group_tag_test.go @@ -19,40 +19,43 @@ import ( "math/rand" "testing" + "github.com/pingcap/kvproto/pkg/coprocessor" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/tikvrpc" ) func TestResourceGroupTagEncoding(t *testing.T) { t.Parallel() sqlDigest := parser.NewDigest(nil) - tag := EncodeResourceGroupTag(sqlDigest, nil) - require.Len(t, tag, 0) + tag := EncodeResourceGroupTag(sqlDigest, nil, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown) + require.Len(t, tag, 2) decodedSQLDigest, err := DecodeResourceGroupTag(tag) require.NoError(t, err) require.Len(t, decodedSQLDigest, 0) sqlDigest = parser.NewDigest([]byte{'a', 'a'}) - tag = EncodeResourceGroupTag(sqlDigest, nil) - // version(1) + prefix(1) + length(1) + content(2hex -> 1byte) - require.Len(t, tag, 4) + tag = EncodeResourceGroupTag(sqlDigest, nil, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown) + // version(1) + prefix(1) + length(1) + content(2hex -> 1byte) + label(2) + require.Len(t, tag, 6) decodedSQLDigest, err = DecodeResourceGroupTag(tag) require.NoError(t, err) require.Equal(t, sqlDigest.Bytes(), decodedSQLDigest) sqlDigest = parser.NewDigest(genRandHex(64)) - tag = EncodeResourceGroupTag(sqlDigest, nil) + tag = EncodeResourceGroupTag(sqlDigest, nil, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown) decodedSQLDigest, err = DecodeResourceGroupTag(tag) require.NoError(t, err) require.Equal(t, sqlDigest.Bytes(), decodedSQLDigest) sqlDigest = parser.NewDigest(genRandHex(510)) - tag = EncodeResourceGroupTag(sqlDigest, nil) + tag = EncodeResourceGroupTag(sqlDigest, nil, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown) decodedSQLDigest, err = DecodeResourceGroupTag(tag) require.NoError(t, err) require.Equal(t, sqlDigest.Bytes(), decodedSQLDigest) @@ -93,6 +96,98 @@ func TestResourceGroupTagEncodingPB(t *testing.T) { require.Nil(t, tag.PlanDigest) } +func TestGetResourceGroupLabelByKey(t *testing.T) { + var label tipb.ResourceGroupTagLabel + // tablecodec.EncodeRowKey(0, []byte{}) + label = GetResourceGroupLabelByKey([]byte{116, 128, 0, 0, 0, 0, 0, 0, 0, 95, 114}) + require.Equal(t, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelRow, label) + // tablecodec.EncodeIndexSeekKey(0, 0, []byte{})) + label = GetResourceGroupLabelByKey([]byte{116, 128, 0, 0, 0, 0, 0, 0, 0, 95, 105, 128, 0, 0, 0, 0, 0, 0, 0}) + require.Equal(t, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelIndex, label) + label = GetResourceGroupLabelByKey([]byte("")) + require.Equal(t, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown, label) +} + +func TestGetFirstKeyFromRequest(t *testing.T) { + var testK1 = []byte("TEST-1") + var testK2 = []byte("TEST-2") + var req *tikvrpc.Request + + require.Nil(t, GetFirstKeyFromRequest(nil)) + + req = &tikvrpc.Request{Req: (*kvrpcpb.GetRequest)(nil)} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.GetRequest{Key: nil}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.GetRequest{Key: testK1}} + require.Equal(t, testK1, GetFirstKeyFromRequest(req)) + + req = &tikvrpc.Request{Req: (*kvrpcpb.BatchGetRequest)(nil)} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.BatchGetRequest{Keys: nil}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.BatchGetRequest{Keys: [][]byte{}}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.BatchGetRequest{Keys: [][]byte{testK2, testK1}}} + require.Equal(t, testK2, GetFirstKeyFromRequest(req)) + + req = &tikvrpc.Request{Req: (*kvrpcpb.ScanRequest)(nil)} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.ScanRequest{StartKey: nil}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.ScanRequest{StartKey: testK1}} + require.Equal(t, testK1, GetFirstKeyFromRequest(req)) + + req = &tikvrpc.Request{Req: (*kvrpcpb.PrewriteRequest)(nil)} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.PrewriteRequest{Mutations: nil}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.PrewriteRequest{Mutations: []*kvrpcpb.Mutation{}}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.PrewriteRequest{Mutations: []*kvrpcpb.Mutation{{Key: testK2}, {Key: testK1}}}} + require.Equal(t, testK2, GetFirstKeyFromRequest(req)) + + req = &tikvrpc.Request{Req: (*kvrpcpb.CommitRequest)(nil)} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.CommitRequest{Keys: nil}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.CommitRequest{Keys: [][]byte{}}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.CommitRequest{Keys: [][]byte{testK1, testK1}}} + require.Equal(t, testK1, GetFirstKeyFromRequest(req)) + + req = &tikvrpc.Request{Req: (*kvrpcpb.BatchRollbackRequest)(nil)} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.BatchRollbackRequest{Keys: nil}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.BatchRollbackRequest{Keys: [][]byte{}}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &kvrpcpb.BatchRollbackRequest{Keys: [][]byte{testK2, testK1}}} + require.Equal(t, testK2, GetFirstKeyFromRequest(req)) + + req = &tikvrpc.Request{Req: (*coprocessor.Request)(nil)} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &coprocessor.Request{Ranges: nil}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &coprocessor.Request{Ranges: []*coprocessor.KeyRange{}}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &coprocessor.Request{Ranges: []*coprocessor.KeyRange{{Start: testK1}}}} + require.Equal(t, testK1, GetFirstKeyFromRequest(req)) + + req = &tikvrpc.Request{Req: (*coprocessor.BatchRequest)(nil)} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &coprocessor.BatchRequest{Regions: nil}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &coprocessor.BatchRequest{Regions: []*coprocessor.RegionInfo{}}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &coprocessor.BatchRequest{Regions: []*coprocessor.RegionInfo{{Ranges: nil}}}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &coprocessor.BatchRequest{Regions: []*coprocessor.RegionInfo{{Ranges: []*coprocessor.KeyRange{}}}}} + require.Nil(t, GetFirstKeyFromRequest(req)) + req = &tikvrpc.Request{Req: &coprocessor.BatchRequest{Regions: []*coprocessor.RegionInfo{{Ranges: []*coprocessor.KeyRange{{Start: testK2}}}}}} + require.Equal(t, testK2, GetFirstKeyFromRequest(req)) +} + func genRandHex(length int) []byte { const chars = "0123456789abcdef" res := make([]byte, length)