diff --git a/build/go-version-check.sh b/build/go-version-check.sh index b285997526d0..719eabe48b59 100755 --- a/build/go-version-check.sh +++ b/build/go-version-check.sh @@ -7,7 +7,8 @@ required_version_major=1 minimum_version_minor=15 -minimum_version_15_patch=3 +minimum_version_15_patch=10 # update to 11 when issue #63836 is addressed +minimum_version_16_patch=3 go=${1-go} diff --git a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go index d7942827e984..3a5b082760ec 100644 --- a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go +++ b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go @@ -596,8 +596,7 @@ func TestClusterRestoreFailCleanup(t *testing.T) { // This test retries the job (by injected a retry error) after restoring a // every system table that has a custom restore function. This tried to tease // out any errors that may occur if some of the system table restoration - // functions are not idempotent (e.g. jobs table), but are retried by the - // restore anyway. + // functions are not idempotent. t.Run("retry-during-custom-system-table-restore", func(t *testing.T) { defer jobs.TestingSetAdoptAndCancelIntervals(100*time.Millisecond, 100*time.Millisecond)() @@ -612,14 +611,16 @@ func TestClusterRestoreFailCleanup(t *testing.T) { _, tcRestore, sqlDBRestore, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) defer cleanupEmptyCluster() - // Inject a retry error + // Inject a retry error, that returns once. + alreadyErrored := false for _, server := range tcRestore.Servers { registry := server.JobRegistry().(*jobs.Registry) registry.TestingResumerCreationKnobs = map[jobspb.Type]func(raw jobs.Resumer) jobs.Resumer{ jobspb.TypeRestore: func(raw jobs.Resumer) jobs.Resumer { r := raw.(*restoreResumer) r.testingKnobs.duringSystemTableRestoration = func(systemTableName string) error { - if systemTableName == customRestoreSystemTable { + if !alreadyErrored && systemTableName == customRestoreSystemTable { + alreadyErrored = true return jobs.NewRetryJobError("injected error") } return nil @@ -629,10 +630,9 @@ func TestClusterRestoreFailCleanup(t *testing.T) { } } - // The initial restore will fail, and restart. + // The initial restore will return an error, and restart. sqlDBRestore.ExpectErr(t, `injected error: restarting in background`, `RESTORE FROM $1`, LocalFoo) - // Expect the job to succeed. If the job fails, it's likely due to - // attempting to restore the same system table data twice. + // Expect the restore to succeed. sqlDBRestore.CheckQueryResultsRetry(t, `SELECT count(*) FROM [SHOW JOBS] WHERE job_type = 'RESTORE' AND status = 'succeeded'`, [][]string{{"1"}}) @@ -841,9 +841,7 @@ func TestReintroduceOfflineSpans(t *testing.T) { const numAccounts = 1000 ctx, _, srcDB, tempDir, cleanupSrc := backupRestoreTestSetupWithParams(t, singleNode, numAccounts, InitManualReplication, params) - _, _, destDB, cleanupDst := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) defer cleanupSrc() - defer cleanupDst() dbBackupLoc := "nodelocal://0/my_db_backup" clusterBackupLoc := "nodelocal://0/my_cluster_backup" @@ -867,7 +865,10 @@ func TestReintroduceOfflineSpans(t *testing.T) { <-dbRestoreStarted srcDB.Exec(t, `BACKUP TO $1 WITH revision_history`, clusterBackupLoc) - // All the restore to finish. This will issue AddSSTable requests at a + var tsMidRestore string + srcDB.QueryRow(t, "SELECT cluster_logical_timestamp()").Scan(&tsMidRestore) + + // Allow the restore to finish. This will issue AddSSTable requests at a // timestamp that is before the last incremental we just took. close(blockDBRestore) @@ -884,16 +885,43 @@ func TestReintroduceOfflineSpans(t *testing.T) { srcDB.Exec(t, `BACKUP TO $1 WITH revision_history`, clusterBackupLoc) - // Restore the incremental backup chain that has missing writes. - destDB.Exec(t, `RESTORE FROM $1 AS OF SYSTEM TIME `+tsBefore, clusterBackupLoc) + t.Run("spans-reintroduced", func(t *testing.T) { + _, _, destDB, cleanupDst := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) + defer cleanupDst() + + // Restore the incremental backup chain that has missing writes. + destDB.Exec(t, `RESTORE FROM $1 AS OF SYSTEM TIME `+tsBefore, clusterBackupLoc) + + // Assert that the restored database has the same number of rows in both the + // source and destination cluster. + checkQuery := `SELECT count(*) FROM restoredb.bank AS OF SYSTEM TIME ` + tsBefore + expectedCount := srcDB.QueryStr(t, checkQuery) + destDB.CheckQueryResults(t, `SELECT count(*) FROM restoredb.bank`, expectedCount) + + checkQuery = `SELECT count(*) FROM restoredb.bank@new_idx AS OF SYSTEM TIME ` + tsBefore + expectedCount = srcDB.QueryStr(t, checkQuery) + destDB.CheckQueryResults(t, `SELECT count(*) FROM restoredb.bank@new_idx`, expectedCount) + }) + + t.Run("restore-canceled", func(t *testing.T) { + defer jobs.TestingSetAdoptAndCancelIntervals(100*time.Millisecond, 100*time.Millisecond)() + _, _, destDB, cleanupDst := backupRestoreTestSetupEmpty(t, singleNode, tempDir, InitManualReplication, base.TestClusterArgs{}) + defer cleanupDst() - // Assert that the restored database has the same number - // of rows in both the source and destination cluster. - checkQuery := `SELECT count(*) FROM restoredb.bank AS OF SYSTEM TIME ` + tsBefore - expectedCount := srcDB.QueryStr(t, checkQuery) - destDB.CheckQueryResults(t, `SELECT count(*) FROM restoredb.bank`, expectedCount) + destDB.Exec(t, `RESTORE FROM $1 AS OF SYSTEM TIME `+tsMidRestore, clusterBackupLoc) - checkQuery = `SELECT count(*) FROM restoredb.bank@new_idx AS OF SYSTEM TIME ` + tsBefore - expectedCount = srcDB.QueryStr(t, checkQuery) - destDB.CheckQueryResults(t, `SELECT count(*) FROM restoredb.bank@new_idx`, expectedCount) + // Wait for the cluster restore job to finish, as well as the restored RESTORE TABLE + // job to cancel. + destDB.CheckQueryResultsRetry(t, ` + SELECT description, status FROM [SHOW JOBS] + WHERE job_type = 'RESTORE' AND status NOT IN ('succeeded', 'canceled')`, + [][]string{}, + ) + // The cluster restore should succeed, but the table restore should have failed. + destDB.CheckQueryResults(t, + `SELECT status, count(*) FROM [SHOW JOBS] WHERE job_type = 'RESTORE' GROUP BY status ORDER BY status`, + [][]string{{"canceled", "1"}, {"succeeded", "1"}}) + + destDB.ExpectErr(t, `relation "restoredb.bank" does not exist`, `SELECT count(*) FROM restoredb.bank`) + }) } diff --git a/pkg/ccl/backupccl/restoration_data.go b/pkg/ccl/backupccl/restoration_data.go index ae82a13b70ec..3b0ecd150308 100644 --- a/pkg/ccl/backupccl/restoration_data.go +++ b/pkg/ccl/backupccl/restoration_data.go @@ -9,6 +9,7 @@ package backupccl import ( + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -107,3 +108,20 @@ func (b *restorationDataBase) isEmpty() bool { // isMainBundle implements restorationData. func (restorationDataBase) isMainBundle() bool { return false } + +// checkForMigratedData checks to see if any of the system tables in the set of +// data that is to be restored has already been restored. If this is the case, +// it is not safe to try and restore the data again since the migration may have +// written to the temporary system table. +func checkForMigratedData(details jobspb.RestoreDetails, dataToRestore restorationData) bool { + for _, systemTable := range dataToRestore.getSystemTables() { + // We only need to check if _any_ of the system tables in this batch of + // data have been migrated. This is because the migration can only + // happen after all of the data in the batch has been restored. + if _, ok := details.SystemTablesMigrated[systemTable.GetName()]; ok { + return true + } + } + + return false +} diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 39f477939aae..8e98951fd568 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -604,6 +604,14 @@ func restore( return emptyRowCount, nil } + // If we've already migrated some of the system tables we're about to + // restore, this implies that a previous attempt restored all of this data. + // We want to avoid restoring again since we'll be shadowing migrated keys. + details := job.Details().(jobspb.RestoreDetails) + if alreadyMigrated := checkForMigratedData(details, dataToRestore); alreadyMigrated { + return emptyRowCount, nil + } + mu := struct { syncutil.Mutex highWaterMark int @@ -1670,8 +1678,7 @@ func (r *restoreResumer) Resume(ctx context.Context, execCtx interface{}) error } // Reload the details as we may have updated the job. details = r.job.Details().(jobspb.RestoreDetails) - } - if details.DescriptorCoverage == tree.AllDescriptors { + if err := r.cleanupTempSystemTables(ctx); err != nil { return err } @@ -2426,8 +2433,8 @@ func (r *restoreResumer) restoreSystemTables( ) error { tempSystemDBID := getTempSystemDBID(restoreDetails) details := r.job.Details().(jobspb.RestoreDetails) - if details.SystemTablesRestored == nil { - details.SystemTablesRestored = make(map[string]bool) + if details.SystemTablesMigrated == nil { + details.SystemTablesMigrated = make(map[string]bool) } // Iterate through all the tables that we're restoring, and if it was restored @@ -2438,18 +2445,36 @@ func (r *restoreResumer) restoreSystemTables( continue } systemTableName := table.GetName() - if details.SystemTablesRestored[systemTableName] { - // We've already restored this table. - continue + stagingTableName := restoreTempSystemDB + "." + systemTableName + + config, ok := systemTableBackupConfiguration[systemTableName] + if !ok { + log.Warningf(ctx, "no configuration specified for table %s... skipping restoration", + systemTableName) + } + + if config.migrationFunc != nil { + if details.SystemTablesMigrated[systemTableName] { + continue + } + + if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if err := config.migrationFunc(ctx, r.execCfg, txn, stagingTableName); err != nil { + return err + } + + // Keep track of which system tables we've migrated so that future job + // restarts don't try to import data over our migrated data. This would + // fail since the restored data would shadow the migrated keys. + details.SystemTablesMigrated[systemTableName] = true + return r.job.SetDetails(ctx, txn, details) + }); err != nil { + return err + } } if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { txn.SetDebugName("system-restore-txn") - config, ok := systemTableBackupConfiguration[systemTableName] - if !ok { - log.Warningf(ctx, "no configuration specified for table %s... skipping restoration", - systemTableName) - } restoreFunc := defaultSystemTableRestoreFunc if config.customRestoreFunc != nil { @@ -2458,15 +2483,11 @@ func (r *restoreResumer) restoreSystemTables( } log.Eventf(ctx, "restoring system table %s", systemTableName) - err := restoreFunc(ctx, r.execCfg, txn, systemTableName, restoreTempSystemDB+"."+systemTableName) + err := restoreFunc(ctx, r.execCfg, txn, systemTableName, stagingTableName) if err != nil { return errors.Wrapf(err, "restoring system table %s", systemTableName) } - - // System table restoration may not be idempotent, so we need to keep - // track of what we've restored. - details.SystemTablesRestored[systemTableName] = true - return r.job.SetDetails(ctx, txn, details) + return nil }); err != nil { return err } diff --git a/pkg/ccl/backupccl/system_schema.go b/pkg/ccl/backupccl/system_schema.go index 1153b3b6cb75..b0b2ea31c100 100644 --- a/pkg/ccl/backupccl/system_schema.go +++ b/pkg/ccl/backupccl/system_schema.go @@ -11,10 +11,16 @@ package backupccl import ( "context" fmt "fmt" + "strings" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -55,6 +61,10 @@ type systemBackupConfiguration struct { // to support the restore (e.g. users that can run the restore, cluster settings // that control how the restore runs, etc...). restoreBeforeData bool + // migrationFunc performs the necessary migrations on the system table data in + // the crdb_temp staging table before it is loaded into the actual system + // table. + migrationFunc func(ctx context.Context, execCtx *sql.ExecutorConfig, txn *kv.Txn, tempTableName string) error // customRestoreFunc is responsible for restoring the data from a table that // holds the restore system table data into the given system table. If none // is provided then `defaultRestoreFunc` is used. @@ -93,6 +103,77 @@ func defaultSystemTableRestoreFunc( // Custom restore functions for different system tables. +// jobsMigrationFunc resets the progress on schema change jobs, and marks all +// other jobs as reverting. +func jobsMigrationFunc( + ctx context.Context, execCfg *sql.ExecutorConfig, txn *kv.Txn, tempTableName string, +) (err error) { + executor := execCfg.InternalExecutor + + const statesToRevert = `('` + string(jobs.StatusRunning) + `', ` + + `'` + string(jobs.StatusPauseRequested) + `', ` + + `'` + string(jobs.StatusPaused) + `')` + + jobsToRevert := make([]int64, 0) + query := `SELECT id, payload FROM ` + tempTableName + ` WHERE status IN ` + statesToRevert + it, err := executor.QueryIteratorEx( + ctx, "restore-fetching-job-payloads", txn, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + query) + if err != nil { + return errors.Wrap(err, "fetching job payloads") + } + defer func() { + closeErr := it.Close() + if err == nil { + err = closeErr + } + }() + for { + ok, err := it.Next(ctx) + if !ok { + if err != nil { + return err + } + break + } + + r := it.Cur() + id, payloadBytes := r[0], r[1] + rawJobID, ok := id.(*tree.DInt) + if !ok { + return errors.Errorf("job: failed to read job id as DInt (was %T)", id) + } + jobID := int64(*rawJobID) + + payload, err := jobs.UnmarshalPayload(payloadBytes) + if err != nil { + return errors.Wrap(err, "failed to unmarshal job to restore") + } + if payload.Type() == jobspb.TypeImport || payload.Type() == jobspb.TypeRestore { + jobsToRevert = append(jobsToRevert, jobID) + } + } + + // Update the status for other jobs. + var updateStatusQuery strings.Builder + fmt.Fprintf(&updateStatusQuery, "UPDATE %s SET status = $1 WHERE id IN ", tempTableName) + fmt.Fprint(&updateStatusQuery, "(") + for i, job := range jobsToRevert { + if i > 0 { + fmt.Fprint(&updateStatusQuery, ", ") + } + fmt.Fprintf(&updateStatusQuery, "'%d'", job) + } + fmt.Fprint(&updateStatusQuery, ")") + + if _, err := executor.Exec(ctx, "updating-job-status", txn, updateStatusQuery.String(), jobs.StatusCancelRequested); err != nil { + return errors.Wrap(err, "updating restored jobs as reverting") + } + + return nil +} + // When restoring the jobs table we don't want to remove existing jobs, since // that includes the restore that we're running. func jobsRestoreFunc( @@ -105,7 +186,7 @@ func jobsRestoreFunc( // When restoring jobs, don't clear the existing table. - restoreQuery := fmt.Sprintf("INSERT INTO system.%s (SELECT * FROM %s);", + restoreQuery := fmt.Sprintf("INSERT INTO system.%s (SELECT * FROM %s) ON CONFLICT DO NOTHING;", systemTableName, tempTableName) opName := systemTableName + "-data-insert" if _, err := executor.Exec(ctx, opName, txn, restoreQuery); err != nil { @@ -178,6 +259,7 @@ var systemTableBackupConfiguration = map[string]systemBackupConfiguration{ }, systemschema.JobsTable.GetName(): { shouldIncludeInClusterBackup: optInToClusterBackup, + migrationFunc: jobsMigrationFunc, customRestoreFunc: jobsRestoreFunc, }, systemschema.ScheduledJobsTable.GetName(): { diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go index 41ae1a40ac98..d2415d39bec2 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -587,12 +587,13 @@ type RestoreDetails struct { // stages have completed via these flags. PrepareCompleted bool `protobuf:"varint,8,opt,name=prepare_completed,json=prepareCompleted,proto3" json:"prepare_completed,omitempty"` StatsInserted bool `protobuf:"varint,9,opt,name=stats_inserted,json=statsInserted,proto3" json:"stats_inserted,omitempty"` - // SystemTablesRestored keeps track of dynamic states that need to happen only - // once during the lifetime of a job. Note, that this state may be shared - // between job versions, so updates to this map must be considered carefully. - // It maps system table names to whether or not they have already been - // restored. - SystemTablesRestored map[string]bool `protobuf:"bytes,17,rep,name=system_tables_restored,json=systemTablesRestored,proto3" json:"system_tables_restored,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + // SystemTablesMigrated keeps track of which system tables data have been + // migrated. We need to keep track of this because if we've modified the + // restored data via a migration, we can't restore back into that span as the + // migrated keys will shadow the ones that will be restored. + // Note, that this state may be shared between job versions, so updates to + // this map must be considered carefully. + SystemTablesMigrated map[string]bool `protobuf:"bytes,17,rep,name=system_tables_migrated,json=systemTablesMigrated,proto3" json:"system_tables_migrated,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` // DescriptorsPublished indicates whether or not the descriptors written in // the job have been transactionally updated after the data was restored. DescriptorsPublished bool `protobuf:"varint,10,opt,name=descriptors_published,json=descriptorsPublished,proto3" json:"descriptors_published,omitempty"` @@ -2574,7 +2575,7 @@ func init() { proto.RegisterType((*BackupProgress)(nil), "cockroach.sql.jobs.jobspb.BackupProgress") proto.RegisterType((*RestoreDetails)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails") proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb.ID]*RestoreDetails_DescriptorRewrite)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails.DescriptorRewritesEntry") - proto.RegisterMapType((map[string]bool)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails.SystemTablesRestoredEntry") + proto.RegisterMapType((map[string]bool)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails.SystemTablesMigratedEntry") proto.RegisterType((*RestoreDetails_DescriptorRewrite)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails.DescriptorRewrite") proto.RegisterType((*RestoreDetails_BackupLocalityInfo)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails.BackupLocalityInfo") proto.RegisterMapType((map[string]string)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails.BackupLocalityInfo.UrisByOriginalLocalityKvEntry") @@ -2624,7 +2625,7 @@ func init() { func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_6c315f3a2536c4ef) } var fileDescriptor_6c315f3a2536c4ef = []byte{ - // 5055 bytes of a gzipped FileDescriptorProto + // 5057 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5b, 0x4b, 0x70, 0x23, 0xc7, 0x79, 0xe6, 0x80, 0x20, 0x1e, 0x3f, 0x1e, 0x1c, 0x34, 0xb9, 0xbb, 0x10, 0x2c, 0x2d, 0x68, 0x58, 0x2b, 0xed, 0xae, 0x24, 0x50, 0xa6, 0x6c, 0x59, 0x5a, 0x4b, 0x2b, 0xe1, 0x45, 0x12, 0xe0, 0xf2, @@ -2710,237 +2711,238 @@ var fileDescriptor_6c315f3a2536c4ef = []byte{ 0xd7, 0xc1, 0x5d, 0xcd, 0xc1, 0xaa, 0x6e, 0x77, 0xba, 0x6d, 0xec, 0x62, 0x83, 0x6d, 0xe8, 0x98, 0x22, 0x8b, 0x82, 0x8a, 0x27, 0xe7, 0xbe, 0x5b, 0x73, 0x69, 0x58, 0x48, 0xb0, 0x43, 0x35, 0xe3, 0x4c, 0x33, 0xc5, 0xa4, 0x75, 0x21, 0x44, 0x87, 0x70, 0x99, 0x1c, 0x12, 0x17, 0x77, 0x54, 0x36, - 0xdd, 0x44, 0x75, 0xb8, 0x05, 0x19, 0xd9, 0x0c, 0x1b, 0x56, 0x65, 0x72, 0x63, 0x6b, 0x32, 0x1c, - 0xb6, 0x8c, 0x44, 0x14, 0x19, 0x3c, 0xf0, 0x99, 0x27, 0xc7, 0x14, 0xa1, 0xb7, 0xe0, 0xd2, 0x60, - 0x67, 0x10, 0xb5, 0xdb, 0x6b, 0xb5, 0x4d, 0xb2, 0x8f, 0x8d, 0x2c, 0xb0, 0x8e, 0xce, 0x07, 0x0a, - 0xb7, 0xbc, 0x32, 0x74, 0x38, 0xb4, 0xd9, 0x75, 0x3a, 0x3b, 0xda, 0x1e, 0xce, 0x26, 0x16, 0xa4, - 0xeb, 0x33, 0xe5, 0xd5, 0xa7, 0xfd, 0x7c, 0x75, 0xe2, 0x9d, 0x4a, 0x70, 0x67, 0xd1, 0x75, 0x30, - 0x0e, 0x6c, 0xfc, 0x8a, 0xc0, 0x0b, 0xee, 0x59, 0x4f, 0x86, 0x14, 0x80, 0x01, 0xe5, 0x67, 0x93, - 0xcf, 0xec, 0x8f, 0x02, 0x28, 0xc8, 0x02, 0xe4, 0xe0, 0x07, 0x5a, 0xdb, 0x34, 0x34, 0x17, 0xab, - 0xa6, 0x65, 0xe0, 0x47, 0x98, 0x64, 0x11, 0x9b, 0xfa, 0x77, 0x27, 0x9f, 0x7a, 0xc5, 0xc7, 0xa8, - 0x53, 0x08, 0x61, 0x69, 0x19, 0x67, 0x58, 0x8c, 0x49, 0xee, 0x7f, 0x25, 0xc8, 0x8c, 0xf1, 0x1c, - 0xda, 0x86, 0x90, 0x7f, 0x40, 0xa2, 0xae, 0x26, 0xc4, 0x0e, 0x47, 0x17, 0xe1, 0xbc, 0x90, 0x69, - 0xa0, 0x3d, 0x88, 0x53, 0x93, 0xb4, 0x5c, 0x7a, 0xfa, 0x0a, 0x31, 0xf0, 0xc6, 0x51, 0x3f, 0x1f, - 0xdb, 0x62, 0xc2, 0x0b, 0x37, 0x11, 0xe3, 0xe0, 0x75, 0x03, 0xe5, 0x21, 0xe1, 0xda, 0x2a, 0x7e, - 0x64, 0x12, 0xd7, 0xb4, 0xf6, 0x58, 0x5c, 0x14, 0x53, 0xc0, 0xb5, 0x6b, 0x42, 0x92, 0xfb, 0xcb, - 0x10, 0xa0, 0x71, 0x2a, 0x44, 0xff, 0x20, 0xc1, 0x8b, 0x5e, 0xb0, 0x64, 0x3b, 0xe6, 0x9e, 0x69, - 0x69, 0xed, 0xa1, 0xa8, 0x49, 0x62, 0xeb, 0xf0, 0xe9, 0x45, 0xf8, 0x56, 0x44, 0x52, 0x9b, 0x02, - 0x7e, 0x34, 0xa2, 0x7a, 0x91, 0x3a, 0x76, 0x1e, 0x51, 0x8d, 0xa9, 0xdc, 0x53, 0xb2, 0xbd, 0x13, - 0x2a, 0xe7, 0xd6, 0xe0, 0xa5, 0x53, 0x81, 0xcf, 0xe3, 0xa6, 0x73, 0x3f, 0x94, 0xe0, 0xca, 0x09, - 0xce, 0x2f, 0x88, 0x93, 0xe2, 0x38, 0x77, 0x83, 0x38, 0x89, 0xa5, 0xef, 0x5e, 0xc0, 0xc1, 0x06, - 0x3b, 0xb1, 0x02, 0x2f, 0x9c, 0x48, 0x20, 0x67, 0x8d, 0x26, 0x16, 0x04, 0xfa, 0x37, 0x09, 0x66, - 0x47, 0xf6, 0x03, 0xfa, 0x24, 0x60, 0xe0, 0x75, 0x7a, 0xbe, 0x65, 0x8d, 0x3c, 0x17, 0x2b, 0x3f, - 0x18, 0xb7, 0xf2, 0x0d, 0xda, 0x02, 0x6b, 0x98, 0xb5, 0xf0, 0xc1, 0x33, 0xb7, 0xc0, 0x21, 0x06, - 0x96, 0xde, 0x08, 0xc7, 0x24, 0x39, 0x54, 0x78, 0x93, 0x0e, 0x90, 0x4d, 0x8f, 0x7f, 0xe0, 0x79, - 0x09, 0x60, 0xdf, 0xdc, 0xdb, 0x57, 0x1f, 0x6a, 0x2e, 0x76, 0x44, 0x6a, 0x21, 0x4e, 0x25, 0x1f, - 0x51, 0x41, 0xe1, 0xcf, 0x01, 0x52, 0xf5, 0x4e, 0xd7, 0x76, 0x5c, 0x2f, 0xea, 0xba, 0x03, 0x11, - 0x4e, 0xf8, 0xc2, 0xc8, 0x8b, 0xa7, 0x2c, 0xe3, 0x50, 0x4d, 0xee, 0xa7, 0x05, 0xc3, 0x08, 0x0c, - 0xb4, 0x09, 0x51, 0xee, 0x1c, 0x49, 0xf6, 0x0a, 0x83, 0x5b, 0x9c, 0x18, 0x8e, 0xbb, 0x59, 0xcf, - 0x37, 0x0a, 0x14, 0x3f, 0x3e, 0x0a, 0x1d, 0x1b, 0x1f, 0xbd, 0x0f, 0x11, 0x9e, 0x98, 0x12, 0x27, - 0xc7, 0xfc, 0x31, 0x47, 0xce, 0xfa, 0xe6, 0xb2, 0xd9, 0xc6, 0xcb, 0x4c, 0xcd, 0xeb, 0x2d, 0xaf, - 0x84, 0x5e, 0x81, 0x18, 0x21, 0xae, 0x4a, 0xcc, 0x1f, 0xf0, 0x88, 0x73, 0x9a, 0xe7, 0x3c, 0x9a, - 0xcd, 0xed, 0xa6, 0xf9, 0x03, 0xac, 0x44, 0x09, 0x71, 0xe9, 0x0f, 0x74, 0x15, 0x98, 0xf7, 0x25, - 0x1a, 0xf5, 0xa9, 0xcc, 0x7d, 0x4e, 0x2b, 0x01, 0x09, 0xc3, 0x39, 0x30, 0xbb, 0xea, 0xee, 0x01, - 0xe1, 0x3e, 0x4b, 0xe0, 0x1c, 0x98, 0xdd, 0xe5, 0x35, 0xa2, 0x44, 0x69, 0xe1, 0xf2, 0x01, 0x41, - 0x39, 0x88, 0x3d, 0xd4, 0xda, 0x6d, 0x16, 0xe1, 0xce, 0x30, 0x14, 0xff, 0x7b, 0x98, 0x24, 0x23, - 0x5f, 0x2d, 0x49, 0x8a, 0x98, 0xb2, 0xab, 0xb9, 0xfb, 0xec, 0x38, 0x12, 0x57, 0x80, 0x8b, 0xb6, - 0x34, 0x77, 0x1f, 0x65, 0x21, 0xca, 0xc7, 0x45, 0xb2, 0xb1, 0x85, 0xe9, 0xeb, 0x49, 0xc5, 0xfb, - 0x44, 0xaf, 0xc2, 0xac, 0xc9, 0x8e, 0xd8, 0xaa, 0x61, 0x3a, 0x58, 0x77, 0xdb, 0x87, 0xcc, 0xdf, - 0xc6, 0x94, 0x34, 0x17, 0x57, 0x85, 0x14, 0xdd, 0x00, 0x79, 0x34, 0x40, 0x61, 0x7e, 0x32, 0xa6, - 0xcc, 0x8e, 0xc4, 0x27, 0x34, 0x96, 0x11, 0x6b, 0x1d, 0x70, 0xfc, 0x59, 0x1e, 0xcb, 0x88, 0x82, - 0x81, 0xd3, 0xbf, 0x01, 0xb2, 0x88, 0x4e, 0x06, 0xba, 0x29, 0x8e, 0xcb, 0xe5, 0x03, 0xd5, 0x22, - 0xcc, 0x75, 0x35, 0x87, 0x60, 0xb5, 0xd5, 0xb3, 0x8c, 0x36, 0x56, 0x39, 0x56, 0x36, 0xcd, 0xb4, - 0x33, 0xac, 0xa8, 0xcc, 0x4a, 0xb8, 0xdd, 0x9d, 0x75, 0x6a, 0xbb, 0xfc, 0xff, 0x70, 0x6a, 0xcb, - 0xfd, 0x3c, 0x04, 0x33, 0x6c, 0x97, 0xa1, 0x5b, 0x10, 0xa6, 0x6b, 0x29, 0xf2, 0x0c, 0x93, 0x46, - 0xce, 0xac, 0x0e, 0x42, 0x10, 0xb6, 0xb4, 0x0e, 0xce, 0x22, 0xb6, 0xd2, 0xec, 0x37, 0xba, 0x02, - 0x51, 0x82, 0xef, 0xab, 0x0f, 0xb4, 0x76, 0x76, 0x8e, 0x19, 0x62, 0x84, 0xe0, 0xfb, 0xf7, 0xb4, - 0x36, 0xba, 0x04, 0x11, 0x93, 0xa8, 0x16, 0x7e, 0x98, 0x9d, 0xe7, 0x7c, 0x6a, 0x92, 0x0d, 0xfc, - 0x10, 0x7d, 0x0d, 0xe2, 0x0f, 0x35, 0xa2, 0xe2, 0x4e, 0xd7, 0x3d, 0x64, 0x73, 0x11, 0xa3, 0xa6, - 0x4b, 0x6a, 0xf4, 0x9b, 0xb9, 0x5d, 0xcd, 0xd9, 0xc3, 0xae, 0xaa, 0xdb, 0x6d, 0x92, 0xbd, 0x44, - 0xb7, 0x2a, 0x8d, 0xda, 0xa9, 0xa8, 0x62, 0xb7, 0x49, 0x23, 0x1c, 0x0b, 0xc9, 0xd3, 0x8d, 0x70, - 0x6c, 0x5a, 0x0e, 0x37, 0xc2, 0xb1, 0xb0, 0x3c, 0xd3, 0x08, 0xc7, 0x66, 0xe4, 0x48, 0x23, 0x1c, - 0x8b, 0xc8, 0xd1, 0x46, 0x38, 0x16, 0x95, 0x63, 0x8d, 0x70, 0x2c, 0x26, 0xc7, 0x1b, 0xe1, 0x58, - 0x5c, 0x86, 0x46, 0x38, 0x06, 0x72, 0xa2, 0x11, 0x8e, 0x25, 0xe4, 0x64, 0x23, 0x1c, 0x4b, 0xca, - 0xa9, 0x46, 0x38, 0x96, 0x92, 0xd3, 0x8d, 0x70, 0x2c, 0x2d, 0xcf, 0x36, 0xc2, 0xb1, 0x59, 0x59, - 0x6e, 0x84, 0x63, 0xb2, 0x9c, 0x69, 0x84, 0x63, 0x19, 0x19, 0xe5, 0x6a, 0x22, 0xb1, 0xa5, 0xa1, - 0xef, 0x0e, 0xcd, 0xd3, 0xc4, 0x11, 0x3d, 0xab, 0x54, 0xf8, 0x85, 0x04, 0x72, 0x13, 0xdf, 0xef, - 0x61, 0x4b, 0xc7, 0xf7, 0xb4, 0x76, 0x65, 0xbf, 0x67, 0x1d, 0xa0, 0x57, 0x60, 0x56, 0xa7, 0x3f, - 0x54, 0x9e, 0xed, 0xa1, 0x33, 0x26, 0xb1, 0x19, 0x4b, 0x31, 0x71, 0x93, 0x4a, 0xe9, 0xc4, 0xbd, - 0x04, 0x20, 0xf4, 0x28, 0x9b, 0xf0, 0x4c, 0x70, 0x9c, 0xab, 0x50, 0x0a, 0x19, 0x81, 0x71, 0xec, - 0x87, 0x8c, 0xb2, 0x86, 0x60, 0x14, 0xfb, 0x21, 0x5a, 0x84, 0x79, 0x0b, 0x3f, 0x72, 0xd5, 0x51, - 0x65, 0x46, 0x4f, 0x4a, 0x86, 0x96, 0x55, 0x82, 0x15, 0x0a, 0xff, 0x1a, 0x82, 0x59, 0xaf, 0xd3, - 0x1e, 0xa7, 0xef, 0x82, 0x4c, 0x57, 0xd7, 0x34, 0x54, 0xd7, 0xe6, 0x48, 0x1e, 0xbb, 0xbf, 0x7f, - 0x0a, 0x1d, 0x8f, 0xa0, 0xd0, 0xef, 0xba, 0xb1, 0x6d, 0xb3, 0xe6, 0x78, 0x10, 0xa0, 0xa4, 0x48, - 0x50, 0x96, 0xdb, 0x81, 0xb4, 0x57, 0x89, 0x4b, 0x50, 0x05, 0x22, 0x43, 0xed, 0xbd, 0x36, 0x41, - 0x7b, 0xde, 0x54, 0x2b, 0xa2, 0x6a, 0xee, 0xf7, 0x01, 0x8d, 0xb7, 0x1d, 0x74, 0xfd, 0x33, 0xdc, - 0xf5, 0x6f, 0x0e, 0x07, 0x20, 0xef, 0x9e, 0x6f, 0x6c, 0x81, 0x6e, 0x07, 0x53, 0x15, 0xff, 0x1c, - 0x82, 0x34, 0x77, 0x4c, 0xbe, 0x4f, 0xa5, 0x14, 0x45, 0x19, 0xd0, 0xb4, 0xf6, 0xd4, 0xae, 0x10, - 0xb2, 0xf1, 0x85, 0x14, 0xd9, 0x2b, 0xf0, 0x95, 0xbf, 0x01, 0x29, 0x07, 0x6b, 0xc6, 0x40, 0x31, - 0xc4, 0x14, 0x93, 0x54, 0xe8, 0x2b, 0x5d, 0x83, 0x34, 0x8b, 0x7b, 0x06, 0x5a, 0xd3, 0x4c, 0x2b, - 0xc5, 0xa4, 0xbe, 0x5a, 0x19, 0x52, 0xa4, 0xab, 0x59, 0x03, 0xad, 0x30, 0x9b, 0xd4, 0x33, 0x12, - 0xab, 0x49, 0x5a, 0x27, 0x18, 0x10, 0x38, 0x98, 0xf4, 0x3a, 0x58, 0xed, 0xda, 0xfc, 0xe4, 0x3d, - 0xad, 0xc4, 0xb9, 0x64, 0xcb, 0x26, 0x68, 0x87, 0x99, 0x0a, 0x9b, 0x0b, 0xd5, 0xe0, 0x93, 0x93, - 0x8d, 0xb0, 0x56, 0x6e, 0x4e, 0x3e, 0x9d, 0xca, 0x2c, 0x19, 0x16, 0x14, 0xfe, 0x56, 0x82, 0x2b, - 0xf4, 0xfc, 0xcd, 0x77, 0x5a, 0x85, 0xdd, 0xe1, 0x78, 0xd6, 0xa9, 0x41, 0x94, 0x9d, 0xe1, 0xfd, - 0x40, 0x6c, 0xf5, 0xa8, 0x9f, 0x8f, 0x50, 0xed, 0x0b, 0x7b, 0xb9, 0x08, 0x05, 0xae, 0xb3, 0x13, - 0xa5, 0xeb, 0x68, 0x16, 0x31, 0xe9, 0xd9, 0x8a, 0x2e, 0x5b, 0x07, 0x77, 0x5a, 0xd8, 0xe1, 0x8b, - 0x91, 0x54, 0xe6, 0x87, 0x0a, 0xd7, 0x79, 0x59, 0x21, 0x07, 0xd9, 0xd1, 0x2e, 0xfb, 0xe9, 0xaa, - 0xdf, 0x81, 0xcb, 0x1b, 0xf8, 0xe1, 0x71, 0xa3, 0x29, 0x43, 0x94, 0x33, 0x9d, 0x67, 0xf2, 0xd7, - 0x47, 0x49, 0x27, 0x78, 0x8d, 0x55, 0x64, 0x3d, 0xdd, 0x66, 0x15, 0x14, 0xaf, 0x62, 0xe1, 0x53, - 0xb8, 0x32, 0x82, 0xee, 0x2f, 0xdf, 0x07, 0x10, 0xa1, 0xe7, 0x74, 0x11, 0x9e, 0xa5, 0xc7, 0x29, - 0x6d, 0x1c, 0xbd, 0x49, 0xf5, 0x15, 0x51, 0xad, 0xa0, 0xb0, 0x3c, 0x5b, 0xaf, 0x83, 0xa9, 0x85, - 0xdc, 0x31, 0x89, 0x8b, 0x3e, 0x84, 0xa4, 0xb0, 0x08, 0x6a, 0x28, 0x5e, 0xb7, 0xcf, 0x30, 0xaa, - 0x84, 0xe3, 0x83, 0x90, 0xc2, 0xdf, 0x49, 0x30, 0x57, 0x75, 0xec, 0x6e, 0x17, 0x1b, 0xc2, 0xe5, - 0xf0, 0xb9, 0xf0, 0x3c, 0x8d, 0x14, 0xf0, 0x34, 0x1b, 0x10, 0xaa, 0x57, 0x45, 0x3c, 0x7c, 0xfb, - 0xa2, 0x61, 0x76, 0xbd, 0x8a, 0xde, 0xe5, 0x13, 0xd2, 0x23, 0x8c, 0x3f, 0xd3, 0x63, 0xe9, 0x96, - 0x21, 0x33, 0x65, 0x8a, 0x8a, 0xa8, 0x50, 0xf8, 0x59, 0x14, 0x2e, 0x05, 0x27, 0x79, 0xa5, 0xe2, - 0x75, 0xfc, 0x33, 0x88, 0x7a, 0x47, 0xee, 0x09, 0x78, 0xf2, 0x38, 0x88, 0xa2, 0x98, 0x8f, 0xe0, - 0xb1, 0xdb, 0xc3, 0x44, 0x4d, 0xc8, 0x98, 0x96, 0x8b, 0x9d, 0x36, 0xd6, 0x1e, 0xd0, 0xe0, 0x82, - 0xce, 0x99, 0x48, 0x71, 0x4e, 0xea, 0xca, 0xe5, 0x00, 0x00, 0x0f, 0x09, 0x3e, 0x83, 0xb9, 0x20, - 0xa8, 0xd7, 0xff, 0xd3, 0x73, 0x6b, 0xac, 0x7b, 0x03, 0x58, 0x2f, 0x09, 0x18, 0x00, 0x12, 0x09, - 0x02, 0xf4, 0xb1, 0x7f, 0x2e, 0xe0, 0xf9, 0xd3, 0x5b, 0xcf, 0x3c, 0x23, 0xd5, 0x91, 0x33, 0xc2, - 0x50, 0xa4, 0xcb, 0x9c, 0xe0, 0x57, 0x14, 0xe9, 0xde, 0x83, 0x08, 0x4f, 0xb1, 0x89, 0x3b, 0x83, - 0xdb, 0xcf, 0x3a, 0x04, 0x9e, 0xba, 0x53, 0x04, 0x5a, 0xee, 0xcf, 0x24, 0x48, 0x06, 0x97, 0x1b, - 0x99, 0x10, 0x63, 0xd3, 0xef, 0x51, 0xda, 0xf4, 0x73, 0x3f, 0xf9, 0x71, 0x53, 0xaa, 0x1b, 0x34, - 0x10, 0x33, 0x1c, 0xbb, 0x3b, 0xb8, 0x33, 0x9a, 0x56, 0x62, 0x54, 0x40, 0x43, 0xc7, 0xdc, 0x1f, - 0x40, 0xdc, 0x9f, 0xf4, 0x40, 0x2e, 0x67, 0xfa, 0x39, 0xe6, 0x72, 0x4e, 0x6d, 0xbf, 0x0a, 0xa9, - 0xa1, 0x19, 0x43, 0x97, 0xfd, 0x3e, 0x84, 0xcb, 0x11, 0xde, 0x87, 0x33, 0x51, 0x0a, 0xbf, 0x8e, - 0xc0, 0xdc, 0x71, 0x4c, 0xfb, 0x09, 0xc8, 0x01, 0xde, 0x52, 0xdb, 0x26, 0x71, 0x85, 0x6d, 0xde, - 0x38, 0x3d, 0xf5, 0x10, 0x20, 0x3f, 0x61, 0x8a, 0x69, 0x67, 0x98, 0x12, 0xbf, 0x07, 0x69, 0x83, - 0x77, 0x5c, 0x64, 0x3f, 0xc5, 0xcd, 0xef, 0x69, 0x87, 0xe1, 0x63, 0x08, 0x50, 0xa0, 0xa7, 0x8c, - 0x40, 0x11, 0x41, 0x3a, 0xa4, 0x7c, 0xf0, 0xc3, 0xae, 0x38, 0x55, 0x5d, 0x9c, 0x0c, 0x93, 0x5e, - 0x2b, 0x14, 0x13, 0xed, 0xc1, 0xac, 0xd7, 0x88, 0x77, 0x00, 0x8f, 0x3f, 0x97, 0x66, 0xbc, 0x89, - 0x69, 0x8a, 0x03, 0xf9, 0x8f, 0x24, 0x98, 0xf3, 0x5a, 0xf2, 0x6f, 0x09, 0x4c, 0x83, 0xd1, 0x59, - 0xaa, 0xdc, 0x3c, 0xea, 0xe7, 0x33, 0x62, 0x66, 0xbc, 0x6b, 0x81, 0x0b, 0xdb, 0x5d, 0xc6, 0x18, - 0x01, 0x34, 0x68, 0x0c, 0x41, 0xcb, 0x69, 0xc3, 0x33, 0x83, 0x18, 0x82, 0x12, 0xdb, 0xc5, 0x63, - 0x08, 0xfa, 0xb3, 0x6e, 0xa0, 0x3f, 0x96, 0x20, 0xc3, 0x2f, 0x2d, 0x3a, 0x3d, 0x57, 0xe3, 0xf7, - 0x83, 0xde, 0xc9, 0xfc, 0x93, 0xa3, 0x7e, 0x7e, 0x96, 0x2d, 0xef, 0xba, 0x28, 0x63, 0xcd, 0x96, - 0x9f, 0xb5, 0xd9, 0x01, 0x8a, 0x38, 0xc8, 0xfa, 0x02, 0x03, 0xad, 0x41, 0x9a, 0xa7, 0x2b, 0xbc, - 0xa7, 0x2d, 0xec, 0xc8, 0x9e, 0x2a, 0xbf, 0xfc, 0xb4, 0x9f, 0x5f, 0x38, 0x66, 0x9f, 0xf0, 0x4c, - 0xc7, 0x3d, 0xae, 0xab, 0xa4, 0x76, 0x83, 0x9f, 0x22, 0x6f, 0x74, 0x19, 0xe6, 0x8f, 0x8d, 0x72, - 0xbe, 0x88, 0xc0, 0xe5, 0x61, 0x22, 0xf4, 0xe3, 0x10, 0x75, 0xd4, 0x43, 0x7e, 0x30, 0x31, 0x99, - 0x7a, 0x18, 0x9c, 0xcc, 0xbc, 0xaf, 0x51, 0x1f, 0xf9, 0xd9, 0x88, 0xbf, 0x79, 0x06, 0x7c, 0xb6, - 0x20, 0x23, 0xf8, 0x9e, 0xd3, 0xf9, 0xd8, 0xf7, 0x05, 0x3c, 0x53, 0xf4, 0xe1, 0x33, 0xc0, 0xb3, - 0xfa, 0xde, 0xa7, 0xef, 0x0d, 0xfe, 0x45, 0x82, 0xd4, 0xd0, 0xc8, 0x7e, 0x9b, 0xee, 0x60, 0xcb, - 0x8f, 0x86, 0xf8, 0x93, 0xa1, 0x77, 0xce, 0x3f, 0xac, 0xe1, 0x20, 0x29, 0xf7, 0xf7, 0x12, 0xa4, - 0x86, 0x26, 0xf2, 0x2b, 0x72, 0x24, 0xcf, 0xbf, 0xe7, 0x2d, 0x48, 0x0f, 0x2f, 0x51, 0xa0, 0x0d, - 0xe9, 0xf9, 0xb4, 0x51, 0xf8, 0x0e, 0x44, 0xb8, 0x04, 0x21, 0x48, 0x7f, 0x54, 0xaa, 0x6f, 0xd7, - 0x37, 0x56, 0xd4, 0xe5, 0x4d, 0x45, 0x5d, 0xa9, 0xc8, 0x53, 0x28, 0x09, 0xb1, 0x6a, 0xed, 0x4e, - 0x8d, 0x0a, 0x65, 0x09, 0x25, 0x20, 0xca, 0xbe, 0x6a, 0x55, 0x39, 0x54, 0x28, 0x83, 0xcc, 0xb1, - 0x77, 0x31, 0x75, 0x0c, 0x34, 0xee, 0x47, 0x45, 0x98, 0x63, 0x41, 0x7a, 0x87, 0xc6, 0x42, 0xd4, - 0x15, 0xaa, 0x81, 0xe8, 0x39, 0xe3, 0x17, 0x51, 0xa7, 0xb8, 0xa1, 0x75, 0x70, 0xe1, 0x97, 0x61, - 0xc8, 0x0c, 0x40, 0x3c, 0xb7, 0xf8, 0x37, 0xd2, 0xe0, 0x04, 0x12, 0x39, 0xf3, 0xbe, 0x68, 0xac, - 0xbe, 0x38, 0x8c, 0x88, 0x2b, 0xee, 0x8f, 0xe8, 0xa6, 0x79, 0xda, 0xcf, 0x67, 0x46, 0x3b, 0x4b, - 0x2e, 0x78, 0xf7, 0xed, 0x75, 0x91, 0xe5, 0x4a, 0x4d, 0xeb, 0x40, 0x1d, 0x3c, 0x5c, 0xe1, 0xb9, - 0x52, 0xd3, 0x3a, 0xd8, 0x51, 0xea, 0x4a, 0x94, 0x16, 0xee, 0x38, 0x26, 0x6a, 0x40, 0xd8, 0xee, - 0xba, 0xde, 0x91, 0xf7, 0xed, 0x73, 0x0d, 0x69, 0xb3, 0x2b, 0xc6, 0xa3, 0x30, 0x0c, 0xd4, 0xe0, - 0x57, 0xa0, 0x83, 0x89, 0x66, 0x14, 0x3a, 0xe1, 0xfb, 0x82, 0xd4, 0xd0, 0x42, 0xe4, 0xf6, 0x20, - 0x19, 0x9c, 0xb1, 0x63, 0xee, 0x45, 0x4a, 0xc3, 0x69, 0x89, 0xd7, 0x26, 0xea, 0xba, 0x38, 0x12, - 0x06, 0xae, 0x2f, 0xbe, 0x03, 0x71, 0x7f, 0x1c, 0xe7, 0xb9, 0xc5, 0xe1, 0x1c, 0xef, 0xe7, 0xdc, - 0x66, 0xe4, 0x48, 0xa1, 0x1f, 0x82, 0xa4, 0x82, 0x89, 0xdd, 0x7e, 0x80, 0x0d, 0x1a, 0xf3, 0xf8, - 0x0f, 0xb5, 0xa4, 0xc9, 0x1f, 0x6a, 0x95, 0x20, 0xee, 0x27, 0x45, 0xcf, 0xf3, 0x58, 0x69, 0x50, - 0x0b, 0xdd, 0x86, 0xaf, 0x05, 0x5f, 0x03, 0xd9, 0x3d, 0xcb, 0xd0, 0x9c, 0x43, 0xd5, 0xc1, 0x9a, - 0xbe, 0x8f, 0x0d, 0x71, 0x5d, 0xf7, 0x42, 0xe0, 0x39, 0x90, 0xd0, 0x50, 0xb8, 0x02, 0xfa, 0x04, - 0x52, 0x7e, 0x25, 0x1a, 0x49, 0xb1, 0x98, 0x23, 0xbd, 0xf4, 0xad, 0xd3, 0xa3, 0x3f, 0x7f, 0xd4, - 0x45, 0x0f, 0x8f, 0x46, 0x4c, 0x4a, 0xb2, 0x15, 0xf8, 0x2a, 0xbc, 0x0f, 0xc9, 0x60, 0x29, 0x8a, - 0x41, 0x78, 0x63, 0x73, 0xa3, 0xc6, 0xf7, 0x74, 0xb9, 0x54, 0x59, 0x5b, 0xae, 0xdf, 0xb9, 0x23, - 0x4b, 0x54, 0x5e, 0xfb, 0xb8, 0xbe, 0x2d, 0x87, 0xe8, 0xee, 0x56, 0x6a, 0xcd, 0xed, 0x92, 0xb2, - 0x2d, 0x4f, 0x17, 0x30, 0xa4, 0x82, 0x2d, 0x51, 0xce, 0xa4, 0x21, 0x26, 0x13, 0x0c, 0x9d, 0xb2, - 0x5f, 0x9d, 0xb0, 0xaf, 0x9e, 0xed, 0x39, 0x41, 0xd4, 0xc2, 0x8f, 0x43, 0x80, 0x06, 0x26, 0x13, - 0x20, 0xe8, 0xd1, 0xc6, 0x42, 0x17, 0x6f, 0x0c, 0xfd, 0xe4, 0xf4, 0x84, 0xf8, 0x34, 0x4b, 0x88, - 0x33, 0xbe, 0xf8, 0xad, 0x26, 0xc5, 0x45, 0x08, 0xf3, 0x3f, 0x61, 0x40, 0x15, 0x07, 0x6b, 0x2e, - 0xa6, 0x7c, 0x4c, 0x4e, 0xcb, 0x40, 0x94, 0x61, 0x86, 0x9f, 0xb8, 0x43, 0xe7, 0x39, 0x71, 0x8b, - 0x49, 0xe1, 0x55, 0xd1, 0xf7, 0x21, 0xa9, 0xdb, 0xed, 0x5e, 0xc7, 0x52, 0xd9, 0xab, 0x09, 0x71, - 0x3c, 0xf8, 0xf6, 0x69, 0x5b, 0x7b, 0xac, 0x73, 0xc5, 0x8a, 0xdd, 0xa6, 0xdf, 0xfe, 0xfb, 0x42, - 0x06, 0xc8, 0x34, 0xd0, 0x8b, 0x10, 0xf7, 0x69, 0x86, 0x99, 0x75, 0x5c, 0x19, 0x08, 0xd0, 0x12, - 0xcc, 0x68, 0x44, 0xb5, 0x77, 0x59, 0xac, 0x7b, 0xd6, 0xbe, 0x53, 0xc2, 0x1a, 0xd9, 0xdc, 0x45, - 0x37, 0x21, 0xd3, 0xd1, 0x1e, 0xa9, 0xbb, 0x8e, 0xa6, 0x8b, 0xd8, 0xb5, 0xcd, 0x69, 0x4f, 0x52, - 0x66, 0x3b, 0xda, 0xa3, 0x65, 0x21, 0xaf, 0x1b, 0x6d, 0x8c, 0xde, 0x82, 0xd4, 0xee, 0x7d, 0x7e, - 0xf2, 0xe1, 0x2e, 0x88, 0x3f, 0x41, 0x99, 0x3d, 0xea, 0xe7, 0x13, 0xcb, 0x77, 0xd9, 0xc4, 0x50, - 0x07, 0xa4, 0x24, 0x76, 0xef, 0xfb, 0x1f, 0xb9, 0xff, 0x96, 0x20, 0x2a, 0x46, 0x84, 0xba, 0x00, - 0x62, 0x7a, 0x4c, 0x83, 0x9b, 0x7a, 0xaa, 0x7c, 0xf7, 0xa8, 0x9f, 0x8f, 0x57, 0x98, 0xb4, 0x5e, - 0x25, 0x4f, 0xfb, 0xf9, 0x0f, 0x9f, 0xd5, 0x7d, 0x78, 0x20, 0x4a, 0x9c, 0x37, 0x52, 0x37, 0x58, - 0x9a, 0x75, 0x5f, 0x23, 0xea, 0xbe, 0x49, 0x5c, 0x7b, 0xcf, 0xd1, 0x3a, 0xe2, 0xfa, 0x37, 0xb9, - 0xaf, 0x91, 0x55, 0x4f, 0x86, 0x72, 0x34, 0x10, 0x7b, 0xc0, 0x1f, 0xbd, 0x70, 0x76, 0xf1, 0xbf, - 0xd1, 0x12, 0x5c, 0xf2, 0x2b, 0xab, 0x74, 0xa6, 0x5a, 0x3d, 0xfd, 0x00, 0x33, 0x87, 0x43, 0x99, - 0x7c, 0xce, 0x2f, 0x5c, 0xd7, 0x1e, 0x95, 0x79, 0x51, 0xe1, 0x12, 0xcc, 0x05, 0x96, 0xd5, 0x0f, - 0x9b, 0x31, 0xc8, 0xeb, 0xe6, 0x9e, 0xa3, 0x05, 0x5f, 0xcc, 0xde, 0x85, 0xd9, 0x91, 0x17, 0xe9, - 0x82, 0x6c, 0x83, 0xe9, 0xc1, 0xe1, 0x27, 0xec, 0xc5, 0x0a, 0xff, 0xf4, 0x42, 0xf7, 0xb4, 0x3e, - 0xf4, 0x5d, 0x98, 0x83, 0x8c, 0xdf, 0x8c, 0xdf, 0xf6, 0x6f, 0x92, 0x10, 0xdd, 0xd2, 0x0e, 0xdb, - 0xb6, 0x66, 0xa0, 0x05, 0x48, 0x78, 0xaf, 0x55, 0xbc, 0xf6, 0xe2, 0x4a, 0x50, 0x34, 0x6c, 0x66, - 0xf2, 0xa8, 0x99, 0x99, 0x90, 0xee, 0x11, 0xec, 0x50, 0x0b, 0x50, 0xd9, 0xf3, 0x79, 0xee, 0x5b, - 0xca, 0xe5, 0xa7, 0xfd, 0xfc, 0xed, 0xc9, 0x16, 0x0f, 0xeb, 0x3d, 0xc7, 0x74, 0x0f, 0x8b, 0xcd, - 0xbb, 0x77, 0x76, 0x04, 0x14, 0xdd, 0xe2, 0xb6, 0x92, 0xea, 0x05, 0x3f, 0xc5, 0xa3, 0x24, 0xba, - 0x10, 0x6a, 0xc7, 0xd4, 0x1d, 0x9b, 0x78, 0xb7, 0x21, 0x42, 0xba, 0xce, 0x84, 0xe8, 0x55, 0x98, - 0xdd, 0x35, 0x2d, 0x76, 0xa1, 0xe7, 0xe9, 0xf1, 0x8b, 0x90, 0xb4, 0x27, 0x16, 0x8a, 0x0f, 0x20, - 0x1d, 0x78, 0x0d, 0x44, 0x8d, 0x30, 0xc2, 0x8c, 0x70, 0xf3, 0xa8, 0x9f, 0x4f, 0x0d, 0x36, 0x35, - 0x37, 0xc4, 0x8b, 0xc4, 0x31, 0xa9, 0x41, 0x33, 0xd4, 0x0c, 0xe7, 0x61, 0x86, 0xfd, 0x71, 0x05, - 0x7f, 0x4e, 0xa9, 0xf0, 0x0f, 0x54, 0x83, 0x94, 0xc8, 0x54, 0xf0, 0xbf, 0xbc, 0x10, 0x4f, 0xa8, - 0x16, 0x02, 0x4b, 0xef, 0xfd, 0x6d, 0x46, 0xb1, 0x66, 0xe9, 0xb6, 0x81, 0x8d, 0x1a, 0xfd, 0x56, - 0x44, 0x62, 0x96, 0x7d, 0x10, 0xb4, 0x02, 0x69, 0xbd, 0x8d, 0x35, 0xab, 0xd7, 0xf5, 0x70, 0xd0, - 0x84, 0x38, 0x29, 0x51, 0x4f, 0x00, 0x6d, 0x00, 0xda, 0x65, 0x6f, 0x5a, 0x82, 0xbd, 0x62, 0x17, - 0x7f, 0x93, 0x80, 0xc9, 0xac, 0xae, 0x32, 0xe8, 0x19, 0x7a, 0x1b, 0x66, 0xda, 0x58, 0x23, 0x58, - 0xbc, 0x95, 0x5d, 0x38, 0x85, 0x06, 0xd9, 0x5f, 0x5f, 0x28, 0x5c, 0x1d, 0xbd, 0x0c, 0x29, 0xcb, - 0xb6, 0x74, 0xcd, 0xd2, 0x71, 0x9b, 0x31, 0x32, 0xbf, 0x63, 0x1c, 0x16, 0xa2, 0x32, 0x44, 0xf8, - 0x6d, 0x34, 0xbb, 0x4b, 0x1f, 0x4f, 0xa8, 0x9f, 0xf8, 0x58, 0x79, 0x75, 0x4a, 0x11, 0x35, 0x51, - 0x0d, 0xa2, 0xe2, 0xfd, 0x1a, 0xbb, 0xa1, 0x3e, 0x33, 0x45, 0x14, 0x78, 0x9d, 0xb2, 0x3a, 0xa5, - 0x78, 0x75, 0xd1, 0xb6, 0xf7, 0x50, 0x90, 0x7b, 0x5d, 0xf1, 0xd6, 0xab, 0x38, 0xe1, 0x31, 0x62, - 0x00, 0x38, 0x84, 0x42, 0x07, 0x68, 0xb2, 0x1b, 0x26, 0x76, 0x77, 0x7d, 0xfa, 0x00, 0x87, 0xde, - 0x48, 0xd0, 0x01, 0xf2, 0x9a, 0x68, 0x03, 0x40, 0xf7, 0x23, 0x01, 0x76, 0xab, 0x9d, 0x58, 0x7a, - 0xfd, 0x3c, 0x41, 0xf2, 0xea, 0x94, 0x12, 0x40, 0x40, 0x77, 0x21, 0xa1, 0x0f, 0xa8, 0x2d, 0x3b, - 0xcb, 0x00, 0xdf, 0x38, 0x97, 0x7f, 0x5b, 0xa5, 0x3e, 0x6d, 0x20, 0x45, 0x9f, 0x42, 0x9a, 0x0c, - 0x1d, 0xaa, 0xb2, 0x97, 0x18, 0xea, 0x9b, 0xe7, 0x4d, 0xc3, 0xae, 0x4e, 0x29, 0x23, 0x48, 0xe8, - 0x77, 0x41, 0x76, 0x47, 0xee, 0x6a, 0xd8, 0xb5, 0xf4, 0xe9, 0x0f, 0xf1, 0x4e, 0xb8, 0x91, 0x5a, - 0x9d, 0x52, 0xc6, 0xd0, 0xd0, 0x67, 0x30, 0x4b, 0x86, 0xff, 0xa8, 0x20, 0x7b, 0x85, 0x35, 0xf0, - 0xcd, 0x53, 0x2f, 0x1c, 0x8e, 0xfb, 0xf3, 0x89, 0xd5, 0x29, 0x65, 0x14, 0x8b, 0xc2, 0x5b, 0xc3, - 0x57, 0x3e, 0xec, 0xd1, 0xc3, 0xe9, 0xf0, 0xc7, 0x5f, 0x41, 0x51, 0xf8, 0x11, 0x2c, 0xb4, 0x06, - 0xf1, 0x8e, 0xe7, 0x2b, 0xb2, 0x2f, 0x9c, 0x79, 0x0e, 0x19, 0x75, 0x5f, 0xab, 0x53, 0xca, 0xa0, - 0x7e, 0x39, 0x0e, 0x51, 0x71, 0x35, 0xe8, 0xdf, 0xdb, 0x47, 0xe5, 0x58, 0xe1, 0x37, 0x31, 0x88, - 0xf9, 0x31, 0xe8, 0x22, 0x20, 0x3f, 0xd4, 0x18, 0xbc, 0x49, 0xa5, 0x2e, 0x28, 0xb4, 0x3a, 0xa5, - 0x64, 0xbc, 0xb2, 0xc1, 0xb3, 0xd4, 0xdb, 0x43, 0x0f, 0x95, 0x26, 0xf9, 0xc3, 0x0f, 0xda, 0x29, - 0xff, 0x25, 0x13, 0x75, 0x0d, 0x1d, 0xdb, 0x30, 0x77, 0xcd, 0x81, 0x6b, 0xe0, 0x89, 0xe4, 0xb4, - 0x27, 0x16, 0xae, 0xe1, 0x1a, 0xa4, 0x9d, 0x9e, 0xc5, 0x6e, 0x01, 0x45, 0x32, 0x80, 0xc7, 0x57, - 0x29, 0x21, 0x15, 0xe7, 0xf9, 0xca, 0x08, 0xeb, 0xdc, 0x38, 0x93, 0x75, 0xbc, 0xb1, 0xaf, 0x4a, - 0x3e, 0xed, 0x2c, 0x8f, 0xd2, 0xce, 0xcd, 0xb3, 0x69, 0x27, 0x00, 0xe3, 0xf3, 0xce, 0xce, 0xb1, - 0xbc, 0xb3, 0x38, 0xe1, 0xc6, 0x09, 0x20, 0x0e, 0x13, 0x4f, 0x65, 0x84, 0x78, 0x6e, 0x9c, 0x49, - 0x3c, 0xc1, 0x31, 0x0a, 0xe6, 0xd9, 0x3c, 0x86, 0x79, 0xde, 0x98, 0x88, 0x79, 0x02, 0x60, 0x41, - 0xea, 0x51, 0x8e, 0xa3, 0x9e, 0xe2, 0x64, 0xd4, 0x13, 0x80, 0x1c, 0xe2, 0x9e, 0xef, 0x8d, 0x71, - 0x8f, 0x7c, 0xf6, 0xe6, 0x3d, 0x36, 0x03, 0xb4, 0x2a, 0x8d, 0x91, 0x8f, 0x76, 0x0c, 0xf9, 0x64, - 0x18, 0xfc, 0x5b, 0xe7, 0x20, 0x9f, 0x40, 0x03, 0xe3, 0xec, 0xf3, 0x31, 0x24, 0x83, 0x8c, 0xc1, - 0xde, 0xee, 0x9c, 0xce, 0x6d, 0x27, 0xfc, 0x05, 0x14, 0xb3, 0x81, 0x40, 0x11, 0xfa, 0xfe, 0x38, - 0xf1, 0xcc, 0x9d, 0x09, 0x7e, 0xc2, 0xed, 0xf4, 0xaa, 0x34, 0xce, 0x3c, 0x77, 0x82, 0xcc, 0x33, - 0x7f, 0xa6, 0x5f, 0x1a, 0x8b, 0x68, 0x57, 0xa5, 0x20, 0xf5, 0x00, 0xc4, 0xbc, 0xc7, 0x0f, 0x01, - 0x1a, 0x2a, 0xfc, 0x54, 0x82, 0xe9, 0x86, 0xdd, 0x42, 0x2f, 0x05, 0x52, 0x93, 0x29, 0x71, 0x14, - 0x9d, 0x69, 0xd8, 0x2d, 0x91, 0x63, 0xfc, 0x60, 0x50, 0x5b, 0x1c, 0xfe, 0xbe, 0x71, 0x4a, 0x57, - 0xfc, 0xcc, 0xae, 0x5f, 0x09, 0xbd, 0x07, 0xd1, 0x2e, 0x0f, 0xae, 0x05, 0x43, 0x15, 0x4e, 0xab, - 0xcf, 0x35, 0x15, 0xaf, 0xca, 0xcd, 0x1b, 0xc1, 0x3f, 0x69, 0x5c, 0xb7, 0x0d, 0x8c, 0xd2, 0x00, - 0x5b, 0x1a, 0x21, 0xdd, 0x7d, 0x47, 0x23, 0x58, 0x9e, 0x42, 0x51, 0x98, 0x5e, 0x5b, 0x6f, 0xca, - 0xd2, 0xcd, 0x8f, 0x83, 0x79, 0xc5, 0xaa, 0x52, 0xaa, 0x6f, 0xd4, 0x37, 0x56, 0xd4, 0x8d, 0xd2, - 0x7a, 0xad, 0x29, 0x4f, 0xa1, 0x2c, 0xcc, 0x7f, 0x54, 0xaa, 0x6f, 0x8b, 0x44, 0xa3, 0x5a, 0xdf, - 0xd8, 0xae, 0x29, 0xf7, 0x4a, 0x77, 0x64, 0x09, 0x5d, 0x06, 0xa4, 0x6c, 0x56, 0xd6, 0x9a, 0xd5, - 0xb2, 0x5a, 0xd9, 0x5c, 0xdf, 0x2a, 0x55, 0xb6, 0xeb, 0x9b, 0x1b, 0x72, 0x08, 0xc5, 0x20, 0x5c, - 0xdd, 0xdc, 0xa8, 0xc9, 0x70, 0xf3, 0x27, 0x61, 0x08, 0xb3, 0x94, 0xc6, 0xcb, 0x90, 0xd8, 0xd9, - 0x68, 0x6e, 0xd5, 0x2a, 0xf5, 0xe5, 0x7a, 0xad, 0x2a, 0x4f, 0xe5, 0xe6, 0x1e, 0x3f, 0x59, 0x98, - 0xa5, 0x45, 0x3b, 0x16, 0xe9, 0x62, 0x9d, 0x71, 0x26, 0xca, 0x41, 0xa4, 0x5c, 0xaa, 0xac, 0xed, - 0x6c, 0xc9, 0x52, 0x2e, 0xfd, 0xf8, 0xc9, 0x02, 0x50, 0x05, 0xce, 0x77, 0xe8, 0x45, 0x9e, 0xf2, - 0xd8, 0x54, 0x6a, 0x72, 0x28, 0x37, 0xfb, 0xf8, 0xc9, 0x42, 0x82, 0x65, 0x52, 0x04, 0x67, 0xbd, - 0x0a, 0xa9, 0x66, 0x65, 0xb5, 0xb6, 0x5e, 0x52, 0x2b, 0xab, 0xa5, 0x8d, 0x95, 0x9a, 0x3c, 0x9d, - 0x9b, 0x7f, 0xfc, 0x64, 0x41, 0x1e, 0xb5, 0x7b, 0xda, 0x44, 0x7d, 0x7d, 0x6b, 0x53, 0xd9, 0x96, - 0xc3, 0x83, 0x26, 0x38, 0xdd, 0xa0, 0x02, 0x00, 0xaf, 0xbd, 0x5c, 0xab, 0x55, 0xe5, 0x99, 0x1c, - 0x7a, 0xfc, 0x64, 0x21, 0x4d, 0xcb, 0x07, 0x2c, 0x82, 0xae, 0x41, 0xb2, 0xa2, 0xd4, 0x4a, 0xdb, - 0x35, 0xb5, 0xb9, 0x5d, 0xda, 0x6e, 0xca, 0x91, 0xc1, 0x48, 0x02, 0xcc, 0x80, 0x8a, 0x90, 0x29, - 0xed, 0x6c, 0x6f, 0xaa, 0x43, 0xba, 0xd1, 0xdc, 0x95, 0xc7, 0x4f, 0x16, 0xe6, 0xa8, 0x6e, 0xa9, - 0xe7, 0xda, 0x41, 0xfd, 0xd7, 0x41, 0x1e, 0xea, 0xbf, 0xba, 0x52, 0x91, 0x63, 0xb9, 0xcb, 0x8f, - 0x9f, 0x2c, 0xa0, 0xd1, 0x21, 0xac, 0x54, 0xd0, 0xb7, 0xe0, 0xf2, 0xf6, 0x27, 0x5b, 0xb5, 0x6a, - 0xad, 0x59, 0x51, 0x87, 0x87, 0x1d, 0xcf, 0x65, 0x1f, 0x3f, 0x59, 0x98, 0xa7, 0x75, 0xc6, 0x86, - 0xfe, 0x06, 0xc8, 0xcd, 0x6d, 0xa5, 0x56, 0x5a, 0x57, 0xeb, 0x1b, 0x2b, 0xb5, 0x26, 0x5b, 0x2c, - 0x18, 0x74, 0x69, 0x64, 0x0f, 0xd3, 0x21, 0x6c, 0xd4, 0x3e, 0x1a, 0xc1, 0x4f, 0x0c, 0xf4, 0x47, - 0xb6, 0x25, 0x5a, 0x80, 0xf8, 0x7a, 0x7d, 0x45, 0x29, 0x31, 0xdc, 0x64, 0x2e, 0xf3, 0xf8, 0xc9, - 0x42, 0x8a, 0xea, 0xf9, 0x9b, 0x2c, 0x17, 0xfb, 0xf1, 0x4f, 0xaf, 0x4e, 0xfd, 0xf5, 0xcf, 0xae, - 0x4e, 0x95, 0xaf, 0x7f, 0xfe, 0x9f, 0x57, 0xa7, 0x3e, 0x3f, 0xba, 0x2a, 0xfd, 0xea, 0xe8, 0xaa, - 0xf4, 0xc5, 0xd1, 0x55, 0xe9, 0x3f, 0x8e, 0xae, 0x4a, 0x7f, 0xf2, 0xe5, 0xd5, 0xa9, 0x5f, 0x7d, - 0x79, 0x75, 0xea, 0x8b, 0x2f, 0xaf, 0x4e, 0x7d, 0x1a, 0xe1, 0x76, 0xdd, 0x8a, 0xb0, 0xb3, 0xe0, - 0x5b, 0xff, 0x17, 0x00, 0x00, 0xff, 0xff, 0xb8, 0x4c, 0x87, 0x4d, 0x83, 0x3e, 0x00, 0x00, + 0xdd, 0x44, 0xed, 0x98, 0x7b, 0x0e, 0x25, 0xe5, 0x6c, 0x86, 0x0d, 0xab, 0x32, 0xb9, 0xb1, 0x35, + 0x19, 0x0e, 0x5b, 0x46, 0xb2, 0x2e, 0x50, 0x78, 0xe0, 0x33, 0x4f, 0x8e, 0x29, 0x42, 0x6f, 0xc1, + 0xa5, 0xc1, 0xce, 0x20, 0x6a, 0xb7, 0xd7, 0x6a, 0x9b, 0x64, 0x1f, 0x1b, 0x59, 0x60, 0x1d, 0x9d, + 0x0f, 0x14, 0x6e, 0x79, 0x65, 0xe8, 0x70, 0x68, 0xb3, 0xeb, 0x74, 0x76, 0xb4, 0x3d, 0x9c, 0x4d, + 0x2c, 0x48, 0xd7, 0x67, 0xca, 0xab, 0x4f, 0xfb, 0xf9, 0xea, 0xc4, 0x3b, 0x95, 0xe0, 0xce, 0xa2, + 0xeb, 0x60, 0x1c, 0xd8, 0xf8, 0x15, 0x81, 0x17, 0xdc, 0xb3, 0x9e, 0x0c, 0x29, 0x00, 0x03, 0xca, + 0xcf, 0x26, 0x9f, 0xd9, 0x1f, 0x05, 0x50, 0x90, 0x05, 0xc8, 0xc1, 0x0f, 0xb4, 0xb6, 0x69, 0x68, + 0x2e, 0x56, 0x4d, 0xcb, 0xc0, 0x8f, 0x30, 0xc9, 0x22, 0x36, 0xf5, 0xef, 0x4e, 0x3e, 0xf5, 0x8a, + 0x8f, 0x51, 0xa7, 0x10, 0xc2, 0xd2, 0x32, 0xce, 0xb0, 0x18, 0x93, 0xdc, 0xff, 0x4a, 0x90, 0x19, + 0xe3, 0x39, 0xb4, 0x0d, 0x21, 0xff, 0x80, 0x44, 0x5d, 0x4d, 0x88, 0x1d, 0x8e, 0x2e, 0xc2, 0x79, + 0x21, 0xd3, 0x40, 0x7b, 0x10, 0xa7, 0x26, 0x69, 0xb9, 0xf4, 0xf4, 0x15, 0x62, 0xe0, 0x8d, 0xa3, + 0x7e, 0x3e, 0xb6, 0xc5, 0x84, 0x17, 0x6e, 0x22, 0xc6, 0xc1, 0xeb, 0x06, 0xca, 0x43, 0xc2, 0xb5, + 0x55, 0xfc, 0xc8, 0x24, 0xae, 0x69, 0xed, 0xb1, 0xb8, 0x28, 0xa6, 0x80, 0x6b, 0xd7, 0x84, 0x24, + 0xf7, 0x97, 0x21, 0x40, 0xe3, 0x54, 0x88, 0xfe, 0x41, 0x82, 0x17, 0xbd, 0x60, 0xc9, 0x76, 0xcc, + 0x3d, 0xd3, 0xd2, 0xda, 0x43, 0x51, 0x93, 0xc4, 0xd6, 0xe1, 0xd3, 0x8b, 0xf0, 0xad, 0x88, 0xa4, + 0x36, 0x05, 0xfc, 0x68, 0x44, 0xf5, 0x22, 0x75, 0xec, 0x3c, 0xa2, 0x1a, 0x53, 0xb9, 0xa7, 0x64, + 0x7b, 0x27, 0x54, 0xce, 0xad, 0xc1, 0x4b, 0xa7, 0x02, 0x9f, 0xc7, 0x4d, 0xe7, 0x7e, 0x28, 0xc1, + 0x95, 0x13, 0x9c, 0x5f, 0x10, 0x27, 0xc5, 0x71, 0xee, 0x06, 0x71, 0x12, 0x4b, 0xdf, 0xbd, 0x80, + 0x83, 0x0d, 0x76, 0x62, 0x05, 0x5e, 0x38, 0x91, 0x40, 0xce, 0x1a, 0x4d, 0x2c, 0x08, 0xf4, 0x6f, + 0x12, 0xcc, 0x8e, 0xec, 0x07, 0xf4, 0x49, 0xc0, 0xc0, 0xeb, 0xf4, 0x7c, 0xcb, 0x1a, 0x79, 0x2e, + 0x56, 0x7e, 0x30, 0x6e, 0xe5, 0x1b, 0xb4, 0x05, 0xd6, 0x30, 0x6b, 0xe1, 0x83, 0x67, 0x6e, 0x81, + 0x43, 0x0c, 0x2c, 0xbd, 0x11, 0x8e, 0x49, 0x72, 0xa8, 0xf0, 0x26, 0x1d, 0x20, 0x9b, 0x59, 0xff, + 0xc0, 0xf3, 0x12, 0xc0, 0xbe, 0xb9, 0xb7, 0xaf, 0x3e, 0xd4, 0x5c, 0xec, 0x88, 0xd4, 0x42, 0x9c, + 0x4a, 0x3e, 0xa2, 0x82, 0xc2, 0x9f, 0x03, 0xa4, 0xea, 0x9d, 0xae, 0xed, 0xb8, 0x5e, 0xd4, 0x75, + 0x07, 0x22, 0x9c, 0xf0, 0x85, 0x91, 0x17, 0x4f, 0x59, 0xc6, 0xa1, 0x9a, 0xdc, 0x4f, 0x0b, 0x86, + 0x11, 0x18, 0x68, 0x13, 0xa2, 0xdc, 0x39, 0x92, 0xec, 0x15, 0x06, 0xb7, 0x38, 0x31, 0x1c, 0x77, + 0xb3, 0x9e, 0x6f, 0x14, 0x28, 0x7e, 0x7c, 0x14, 0x3a, 0x36, 0x3e, 0x7a, 0x1f, 0x22, 0x3c, 0x31, + 0x25, 0x4e, 0x8e, 0xf9, 0x63, 0x8e, 0x9c, 0xf5, 0xcd, 0x65, 0xb3, 0x8d, 0x97, 0x99, 0x9a, 0xd7, + 0x5b, 0x5e, 0x09, 0xbd, 0x02, 0x31, 0x42, 0x5c, 0x95, 0x98, 0x3f, 0xe0, 0x11, 0xe7, 0x34, 0xcf, + 0x79, 0x34, 0x9b, 0xdb, 0x4d, 0xf3, 0x07, 0x58, 0x89, 0x12, 0xe2, 0xd2, 0x1f, 0xe8, 0x2a, 0x30, + 0xef, 0x4b, 0x34, 0xea, 0x53, 0x99, 0xfb, 0x9c, 0x56, 0x02, 0x12, 0x86, 0x73, 0x60, 0x76, 0xd5, + 0xdd, 0x03, 0xc2, 0x7d, 0x96, 0xc0, 0x39, 0x30, 0xbb, 0xcb, 0x6b, 0x44, 0x89, 0xd2, 0xc2, 0xe5, + 0x03, 0x82, 0x72, 0x10, 0x7b, 0xa8, 0xb5, 0xdb, 0x2c, 0xc2, 0x9d, 0x61, 0x28, 0xfe, 0xf7, 0x30, + 0x49, 0x46, 0xbe, 0x5a, 0x92, 0x14, 0x31, 0x65, 0x57, 0x73, 0xf7, 0xd9, 0x71, 0x24, 0xae, 0x00, + 0x17, 0x6d, 0x69, 0xee, 0x3e, 0xca, 0x42, 0x94, 0x8f, 0x8b, 0x64, 0x63, 0x0b, 0xd3, 0xd7, 0x93, + 0x8a, 0xf7, 0x89, 0x5e, 0x85, 0x59, 0x93, 0x1d, 0xb1, 0x55, 0xc3, 0x74, 0xb0, 0xee, 0xb6, 0x0f, + 0x99, 0xbf, 0x8d, 0x29, 0x69, 0x2e, 0xae, 0x0a, 0x29, 0xba, 0x01, 0xf2, 0x68, 0x80, 0xc2, 0xfc, + 0x64, 0x4c, 0x99, 0x1d, 0x89, 0x4f, 0x68, 0x2c, 0x23, 0xd6, 0x3a, 0xe0, 0xf8, 0xb3, 0x3c, 0x96, + 0x11, 0x05, 0x03, 0xa7, 0x7f, 0x03, 0x64, 0x11, 0x9d, 0x0c, 0x74, 0x53, 0x1c, 0x97, 0xcb, 0x07, + 0xaa, 0x45, 0x98, 0xeb, 0x6a, 0x0e, 0xc1, 0x6a, 0xab, 0x67, 0x19, 0x6d, 0xac, 0x72, 0xac, 0x6c, + 0x9a, 0x69, 0x67, 0x58, 0x51, 0x99, 0x95, 0x70, 0xbb, 0x3b, 0xeb, 0xd4, 0x76, 0xf9, 0xff, 0xe1, + 0xd4, 0x96, 0xfb, 0x79, 0x08, 0x66, 0xd8, 0x2e, 0x43, 0xb7, 0x20, 0x4c, 0xd7, 0x52, 0xe4, 0x19, + 0x26, 0x8d, 0x9c, 0x59, 0x1d, 0x84, 0x20, 0x6c, 0x69, 0x1d, 0x9c, 0x45, 0x6c, 0xa5, 0xd9, 0x6f, + 0x74, 0x05, 0xa2, 0x04, 0xdf, 0x57, 0x1f, 0x68, 0xed, 0xec, 0x1c, 0x33, 0xc4, 0x08, 0xc1, 0xf7, + 0xef, 0x69, 0x6d, 0x74, 0x09, 0x22, 0x26, 0x51, 0x2d, 0xfc, 0x30, 0x3b, 0xcf, 0xf9, 0xd4, 0x24, + 0x1b, 0xf8, 0x21, 0xfa, 0x1a, 0xc4, 0x1f, 0x6a, 0x44, 0xc5, 0x9d, 0xae, 0x7b, 0xc8, 0xe6, 0x22, + 0x46, 0x4d, 0x97, 0xd4, 0xe8, 0x37, 0x73, 0xbb, 0x9a, 0xb3, 0x87, 0x5d, 0x55, 0xb7, 0xdb, 0x24, + 0x7b, 0x89, 0x6e, 0x55, 0x1a, 0xb5, 0x53, 0x51, 0xc5, 0x6e, 0x93, 0x46, 0x38, 0x16, 0x92, 0xa7, + 0x1b, 0xe1, 0xd8, 0xb4, 0x1c, 0x6e, 0x84, 0x63, 0x61, 0x79, 0xa6, 0x11, 0x8e, 0xcd, 0xc8, 0x91, + 0x46, 0x38, 0x16, 0x91, 0xa3, 0x8d, 0x70, 0x2c, 0x2a, 0xc7, 0x1a, 0xe1, 0x58, 0x4c, 0x8e, 0x37, + 0xc2, 0xb1, 0xb8, 0x0c, 0x8d, 0x70, 0x0c, 0xe4, 0x44, 0x23, 0x1c, 0x4b, 0xc8, 0xc9, 0x46, 0x38, + 0x96, 0x94, 0x53, 0x8d, 0x70, 0x2c, 0x25, 0xa7, 0x1b, 0xe1, 0x58, 0x5a, 0x9e, 0x6d, 0x84, 0x63, + 0xb3, 0xb2, 0xdc, 0x08, 0xc7, 0x64, 0x39, 0xd3, 0x08, 0xc7, 0x32, 0x32, 0xca, 0xd5, 0x44, 0x62, + 0x4b, 0x43, 0xdf, 0x1d, 0x9a, 0xa7, 0x89, 0x23, 0x7a, 0x56, 0xa9, 0xf0, 0x0b, 0x09, 0xe4, 0x26, + 0xbe, 0xdf, 0xc3, 0x96, 0x8e, 0xef, 0x69, 0xed, 0xca, 0x7e, 0xcf, 0x3a, 0x40, 0xaf, 0xc0, 0xac, + 0x4e, 0x7f, 0xa8, 0x3c, 0xdb, 0x43, 0x67, 0x4c, 0x62, 0x33, 0x96, 0x62, 0xe2, 0x26, 0x95, 0xd2, + 0x89, 0x7b, 0x09, 0x40, 0xe8, 0x51, 0x36, 0xe1, 0x99, 0xe0, 0x38, 0x57, 0xa1, 0x14, 0x32, 0x02, + 0xe3, 0xd8, 0x0f, 0x19, 0x65, 0x0d, 0xc1, 0x28, 0xf6, 0x43, 0xb4, 0x08, 0xf3, 0x16, 0x7e, 0xe4, + 0xaa, 0xa3, 0xca, 0x8c, 0x9e, 0x94, 0x0c, 0x2d, 0xab, 0x04, 0x2b, 0x14, 0xfe, 0x35, 0x04, 0xb3, + 0x5e, 0xa7, 0x3d, 0x4e, 0xdf, 0x05, 0x99, 0xae, 0xae, 0x69, 0xa8, 0xae, 0xcd, 0x91, 0x3c, 0x76, + 0x7f, 0xff, 0x14, 0x3a, 0x1e, 0x41, 0xa1, 0xdf, 0x75, 0x63, 0xdb, 0x66, 0xcd, 0xf1, 0x20, 0x40, + 0x49, 0x91, 0xa0, 0x2c, 0xb7, 0x03, 0x69, 0xaf, 0x12, 0x97, 0xa0, 0x0a, 0x44, 0x86, 0xda, 0x7b, + 0x6d, 0x82, 0xf6, 0xbc, 0xa9, 0x56, 0x44, 0xd5, 0xdc, 0xef, 0x03, 0x1a, 0x6f, 0x3b, 0xe8, 0xfa, + 0x67, 0xb8, 0xeb, 0xdf, 0x1c, 0x0e, 0x40, 0xde, 0x3d, 0xdf, 0xd8, 0x02, 0xdd, 0x0e, 0xa6, 0x2a, + 0xfe, 0x39, 0x04, 0x69, 0xee, 0x98, 0x7c, 0x9f, 0x4a, 0x29, 0x8a, 0x32, 0xa0, 0x69, 0xed, 0xa9, + 0x5d, 0x21, 0x64, 0xe3, 0x0b, 0x29, 0xb2, 0x57, 0xe0, 0x2b, 0x7f, 0x03, 0x52, 0x0e, 0xd6, 0x8c, + 0x81, 0x62, 0x88, 0x29, 0x26, 0xa9, 0xd0, 0x57, 0xba, 0x06, 0x69, 0x16, 0xf7, 0x0c, 0xb4, 0xa6, + 0x99, 0x56, 0x8a, 0x49, 0x7d, 0xb5, 0x32, 0xa4, 0x48, 0x57, 0xb3, 0x06, 0x5a, 0x61, 0x36, 0xa9, + 0x67, 0x24, 0x56, 0x93, 0xb4, 0x4e, 0x30, 0x20, 0x70, 0x30, 0xe9, 0x75, 0xb0, 0xda, 0xb5, 0xf9, + 0xc9, 0x7b, 0x5a, 0x89, 0x73, 0xc9, 0x96, 0x4d, 0xd0, 0x0e, 0x33, 0x15, 0x36, 0x17, 0xaa, 0xc1, + 0x27, 0x27, 0x1b, 0x61, 0xad, 0xdc, 0x9c, 0x7c, 0x3a, 0x95, 0x59, 0x32, 0x2c, 0x28, 0xfc, 0xad, + 0x04, 0x57, 0xe8, 0xf9, 0x9b, 0xef, 0xb4, 0x0a, 0xbb, 0xc3, 0xf1, 0xac, 0x53, 0x83, 0x28, 0x3b, + 0xc3, 0xfb, 0x81, 0xd8, 0xea, 0x51, 0x3f, 0x1f, 0xa1, 0xda, 0x17, 0xf6, 0x72, 0x11, 0x0a, 0x5c, + 0x67, 0x27, 0x4a, 0xd7, 0xd1, 0x2c, 0x62, 0xd2, 0xb3, 0x15, 0x5d, 0xb6, 0x0e, 0xee, 0xb4, 0xb0, + 0xc3, 0x17, 0x23, 0xa9, 0xcc, 0x0f, 0x15, 0xae, 0xf3, 0xb2, 0x42, 0x0e, 0xb2, 0xa3, 0x5d, 0xf6, + 0xd3, 0x55, 0xbf, 0x03, 0x97, 0x37, 0xf0, 0xc3, 0xe3, 0x46, 0x53, 0x86, 0x28, 0x67, 0x3a, 0xcf, + 0xe4, 0xaf, 0x8f, 0x92, 0x4e, 0xf0, 0x1a, 0xab, 0xc8, 0x7a, 0xba, 0xcd, 0x2a, 0x28, 0x5e, 0xc5, + 0xc2, 0xa7, 0x70, 0x65, 0x04, 0xdd, 0x5f, 0xbe, 0x0f, 0x20, 0x42, 0xcf, 0xe9, 0x22, 0x3c, 0x4b, + 0x8f, 0x53, 0xda, 0x38, 0x7a, 0x93, 0xea, 0x2b, 0xa2, 0x5a, 0x41, 0x61, 0x79, 0xb6, 0x5e, 0x07, + 0x53, 0x0b, 0xb9, 0x63, 0x12, 0x17, 0x7d, 0x08, 0x49, 0x61, 0x11, 0xd4, 0x50, 0xbc, 0x6e, 0x9f, + 0x61, 0x54, 0x09, 0xc7, 0x07, 0x21, 0x85, 0xbf, 0x93, 0x60, 0xae, 0xea, 0xd8, 0xdd, 0x2e, 0x36, + 0x84, 0xcb, 0xe1, 0x73, 0xe1, 0x79, 0x1a, 0x29, 0xe0, 0x69, 0x36, 0x20, 0x54, 0xaf, 0x8a, 0x78, + 0xf8, 0xf6, 0x45, 0xc3, 0xec, 0x7a, 0x15, 0xbd, 0xcb, 0x27, 0xa4, 0x47, 0x18, 0x7f, 0xa6, 0xc7, + 0xd2, 0x2d, 0x43, 0x66, 0xca, 0x14, 0x15, 0x51, 0xa1, 0xf0, 0xb3, 0x28, 0x5c, 0x0a, 0x4e, 0xf2, + 0x4a, 0xc5, 0xeb, 0xf8, 0x67, 0x10, 0xf5, 0x8e, 0xdc, 0x13, 0xf0, 0xe4, 0x71, 0x10, 0x45, 0x31, + 0x1f, 0xc1, 0x63, 0xb7, 0x87, 0x89, 0x9a, 0x90, 0x31, 0x2d, 0x17, 0x3b, 0x6d, 0xac, 0x3d, 0xa0, + 0xc1, 0x05, 0x9d, 0x33, 0x91, 0xe2, 0x9c, 0xd4, 0x95, 0xcb, 0x01, 0x00, 0x1e, 0x12, 0x7c, 0x06, + 0x73, 0x41, 0x50, 0xaf, 0xff, 0xa7, 0xe7, 0xd6, 0x58, 0xf7, 0x06, 0xb0, 0x5e, 0x12, 0x30, 0x00, + 0x24, 0x12, 0x04, 0xe8, 0x63, 0xff, 0x5c, 0xc0, 0xf3, 0xa7, 0xb7, 0x9e, 0x79, 0x46, 0xaa, 0x23, + 0x67, 0x84, 0xa1, 0x48, 0x97, 0x39, 0xc1, 0xaf, 0x28, 0xd2, 0xbd, 0x07, 0x11, 0x9e, 0x62, 0x13, + 0x77, 0x06, 0xb7, 0x9f, 0x75, 0x08, 0x3c, 0x75, 0xa7, 0x08, 0xb4, 0xdc, 0x9f, 0x49, 0x90, 0x0c, + 0x2e, 0x37, 0x32, 0x21, 0xc6, 0xa6, 0xdf, 0xa3, 0xb4, 0xe9, 0xe7, 0x7e, 0xf2, 0xe3, 0xa6, 0x54, + 0x37, 0x68, 0x20, 0x66, 0x38, 0x76, 0x77, 0x70, 0x67, 0x34, 0xad, 0xc4, 0xa8, 0x80, 0x86, 0x8e, + 0xb9, 0x3f, 0x80, 0xb8, 0x3f, 0xe9, 0x81, 0x5c, 0xce, 0xf4, 0x73, 0xcc, 0xe5, 0x9c, 0xda, 0x7e, + 0x15, 0x52, 0x43, 0x33, 0x86, 0x2e, 0xfb, 0x7d, 0x08, 0x97, 0x23, 0xbc, 0x0f, 0x67, 0xa2, 0x14, + 0x7e, 0x1d, 0x81, 0xb9, 0xe3, 0x98, 0xf6, 0x13, 0x90, 0x03, 0xbc, 0xa5, 0xb6, 0x4d, 0xe2, 0x0a, + 0xdb, 0xbc, 0x71, 0x7a, 0xea, 0x21, 0x40, 0x7e, 0xc2, 0x14, 0xd3, 0xce, 0x30, 0x25, 0x7e, 0x0f, + 0xd2, 0x06, 0xef, 0xb8, 0xc8, 0x7e, 0x8a, 0x9b, 0xdf, 0xd3, 0x0e, 0xc3, 0xc7, 0x10, 0xa0, 0x40, + 0x4f, 0x19, 0x81, 0x22, 0x82, 0x74, 0x48, 0xf9, 0xe0, 0x87, 0x5d, 0x71, 0xaa, 0xba, 0x38, 0x19, + 0x26, 0xbd, 0x56, 0x28, 0x26, 0xda, 0x83, 0x59, 0xaf, 0x11, 0xef, 0x00, 0x1e, 0x7f, 0x2e, 0xcd, + 0x78, 0x13, 0xd3, 0x14, 0x07, 0xf2, 0x1f, 0x49, 0x30, 0xe7, 0xb5, 0xe4, 0xdf, 0x12, 0x98, 0x06, + 0xa3, 0xb3, 0x54, 0xb9, 0x79, 0xd4, 0xcf, 0x67, 0xc4, 0xcc, 0x78, 0xd7, 0x02, 0x17, 0xb6, 0xbb, + 0x8c, 0x31, 0x02, 0x68, 0xd0, 0x18, 0x82, 0x96, 0xd3, 0x86, 0x67, 0x06, 0x31, 0x04, 0x25, 0xb6, + 0x8b, 0xc7, 0x10, 0xf4, 0x67, 0xdd, 0x40, 0x7f, 0x2c, 0x41, 0x86, 0x5f, 0x5a, 0x74, 0x7a, 0xae, + 0xc6, 0xef, 0x07, 0xbd, 0x93, 0xf9, 0x27, 0x47, 0xfd, 0xfc, 0x2c, 0x5b, 0xde, 0x75, 0x51, 0xc6, + 0x9a, 0x2d, 0x3f, 0x6b, 0xb3, 0x03, 0x14, 0x71, 0x90, 0xf5, 0x05, 0x06, 0x5a, 0x83, 0x34, 0x4f, + 0x57, 0x78, 0x4f, 0x5b, 0xd8, 0x91, 0x3d, 0x55, 0x7e, 0xf9, 0x69, 0x3f, 0xbf, 0x70, 0xcc, 0x3e, + 0xe1, 0x99, 0x8e, 0x7b, 0x5c, 0x57, 0x49, 0xed, 0x06, 0x3f, 0x45, 0xde, 0xe8, 0x32, 0xcc, 0x1f, + 0x1b, 0xe5, 0x7c, 0x11, 0x81, 0xcb, 0xc3, 0x44, 0xe8, 0xc7, 0x21, 0xea, 0xa8, 0x87, 0xfc, 0x60, + 0x62, 0x32, 0xf5, 0x30, 0x38, 0x99, 0x79, 0x5f, 0xa3, 0x3e, 0xf2, 0xb3, 0x11, 0x7f, 0xf3, 0x0c, + 0xf8, 0x6c, 0x41, 0x46, 0xf0, 0x3d, 0xa7, 0xf3, 0xb1, 0xef, 0x0b, 0x78, 0xa6, 0xe8, 0xc3, 0x67, + 0x80, 0x67, 0xf5, 0xbd, 0x4f, 0xdf, 0x1b, 0xfc, 0x8b, 0x04, 0xa9, 0xa1, 0x91, 0xfd, 0x36, 0xdd, + 0xc1, 0x96, 0x1f, 0x0d, 0xf1, 0x27, 0x43, 0xef, 0x9c, 0x7f, 0x58, 0xc3, 0x41, 0x52, 0xee, 0xef, + 0x25, 0x48, 0x0d, 0x4d, 0xe4, 0x57, 0xe4, 0x48, 0x9e, 0x7f, 0xcf, 0x5b, 0x90, 0x1e, 0x5e, 0xa2, + 0x40, 0x1b, 0xd2, 0xf3, 0x69, 0xa3, 0xf0, 0x1d, 0x88, 0x70, 0x09, 0x42, 0x90, 0xfe, 0xa8, 0x54, + 0xdf, 0xae, 0x6f, 0xac, 0xa8, 0xcb, 0x9b, 0x8a, 0xba, 0x52, 0x91, 0xa7, 0x50, 0x12, 0x62, 0xd5, + 0xda, 0x9d, 0x1a, 0x15, 0xca, 0x12, 0x4a, 0x40, 0x94, 0x7d, 0xd5, 0xaa, 0x72, 0xa8, 0x50, 0x06, + 0x99, 0x63, 0xef, 0x62, 0xea, 0x18, 0x68, 0xdc, 0x8f, 0x8a, 0x30, 0xc7, 0x82, 0xf4, 0x0e, 0x8d, + 0x85, 0xa8, 0x2b, 0x54, 0x03, 0xd1, 0x73, 0xc6, 0x2f, 0xa2, 0x4e, 0x71, 0x43, 0xeb, 0xe0, 0xc2, + 0x2f, 0xc3, 0x90, 0x19, 0x80, 0x78, 0x6e, 0xf1, 0x6f, 0xa4, 0xc1, 0x09, 0x24, 0x72, 0xe6, 0x7d, + 0xd1, 0x58, 0x7d, 0x71, 0x18, 0x11, 0x57, 0xdc, 0x1f, 0xd1, 0x4d, 0xf3, 0xb4, 0x9f, 0xcf, 0x8c, + 0x76, 0x96, 0x5c, 0xf0, 0xee, 0xdb, 0xeb, 0x22, 0xcb, 0x95, 0x9a, 0xd6, 0x81, 0x3a, 0x78, 0xb8, + 0xc2, 0x73, 0xa5, 0xa6, 0x75, 0xb0, 0xa3, 0xd4, 0x95, 0x28, 0x2d, 0xdc, 0x71, 0x4c, 0xd4, 0x80, + 0xb0, 0xdd, 0x75, 0xbd, 0x23, 0xef, 0xdb, 0xe7, 0x1a, 0xd2, 0x66, 0x57, 0x8c, 0x47, 0x61, 0x18, + 0xa8, 0xc1, 0xaf, 0x40, 0x07, 0x13, 0xcd, 0x28, 0x74, 0xc2, 0xf7, 0x05, 0xa9, 0xa1, 0x85, 0xc8, + 0xed, 0x41, 0x32, 0x38, 0x63, 0xc7, 0xdc, 0x8b, 0x94, 0x86, 0xd3, 0x12, 0xaf, 0x4d, 0xd4, 0x75, + 0x71, 0x24, 0x0c, 0x5c, 0x5f, 0x7c, 0x07, 0xe2, 0xfe, 0x38, 0xce, 0x73, 0x8b, 0xc3, 0x39, 0xde, + 0xcf, 0xb9, 0xcd, 0xc8, 0x91, 0x42, 0x3f, 0x04, 0x49, 0x05, 0x13, 0xbb, 0xfd, 0x00, 0x1b, 0x34, + 0xe6, 0xf1, 0x1f, 0x6a, 0x49, 0x93, 0x3f, 0xd4, 0x2a, 0x41, 0xdc, 0x4f, 0x8a, 0x9e, 0xe7, 0xb1, + 0xd2, 0xa0, 0x16, 0xba, 0x0d, 0x5f, 0x0b, 0xbe, 0x06, 0xb2, 0x7b, 0x96, 0xa1, 0x39, 0x87, 0xaa, + 0x83, 0x35, 0x7d, 0x1f, 0x1b, 0xe2, 0xba, 0xee, 0x85, 0xc0, 0x73, 0x20, 0xa1, 0xa1, 0x70, 0x05, + 0xf4, 0x09, 0xa4, 0xfc, 0x4a, 0x34, 0x92, 0x62, 0x31, 0x47, 0x7a, 0xe9, 0x5b, 0xa7, 0x47, 0x7f, + 0xfe, 0xa8, 0x8b, 0x1e, 0x1e, 0x8d, 0x98, 0x94, 0x64, 0x2b, 0xf0, 0x55, 0x78, 0x1f, 0x92, 0xc1, + 0x52, 0x14, 0x83, 0xf0, 0xc6, 0xe6, 0x46, 0x8d, 0xef, 0xe9, 0x72, 0xa9, 0xb2, 0xb6, 0x5c, 0xbf, + 0x73, 0x47, 0x96, 0xa8, 0xbc, 0xf6, 0x71, 0x7d, 0x5b, 0x0e, 0xd1, 0xdd, 0xad, 0xd4, 0x9a, 0xdb, + 0x25, 0x65, 0x5b, 0x9e, 0x2e, 0x60, 0x48, 0x05, 0x5b, 0xa2, 0x9c, 0x49, 0x43, 0x4c, 0x26, 0x18, + 0x3a, 0x65, 0xbf, 0x3a, 0x61, 0x5f, 0x3d, 0xdb, 0x73, 0x82, 0xa8, 0x85, 0x1f, 0x87, 0x00, 0x0d, + 0x4c, 0x26, 0x40, 0xd0, 0xa3, 0x8d, 0x85, 0x2e, 0xde, 0x18, 0xfa, 0xc9, 0xe9, 0x09, 0xf1, 0x69, + 0x96, 0x10, 0x67, 0x7c, 0xf1, 0x5b, 0x4d, 0x8a, 0x8b, 0x10, 0xe6, 0x7f, 0xc2, 0x80, 0x2a, 0x0e, + 0xd6, 0x5c, 0x4c, 0xf9, 0x98, 0x9c, 0x96, 0x81, 0x28, 0xc3, 0x0c, 0x3f, 0x71, 0x87, 0xce, 0x73, + 0xe2, 0x16, 0x93, 0xc2, 0xab, 0xa2, 0xef, 0x43, 0x52, 0xb7, 0xdb, 0xbd, 0x8e, 0xa5, 0xb2, 0x57, + 0x13, 0xe2, 0x78, 0xf0, 0xed, 0xd3, 0xb6, 0xf6, 0x58, 0xe7, 0x8a, 0x15, 0xbb, 0x4d, 0xbf, 0xfd, + 0xf7, 0x85, 0x0c, 0x90, 0x69, 0xa0, 0x17, 0x21, 0xee, 0xd3, 0x0c, 0x33, 0xeb, 0xb8, 0x32, 0x10, + 0xa0, 0x25, 0x98, 0xd1, 0x88, 0x6a, 0xef, 0xb2, 0x58, 0xf7, 0xac, 0x7d, 0xa7, 0x84, 0x35, 0xb2, + 0xb9, 0x8b, 0x6e, 0x42, 0xa6, 0xa3, 0x3d, 0x52, 0x77, 0x1d, 0x4d, 0x17, 0xb1, 0x6b, 0x9b, 0xd3, + 0x9e, 0xa4, 0xcc, 0x76, 0xb4, 0x47, 0xcb, 0x42, 0x5e, 0x37, 0xda, 0x18, 0xbd, 0x05, 0xa9, 0xdd, + 0xfb, 0xfc, 0xe4, 0xc3, 0x5d, 0x10, 0x7f, 0x82, 0x32, 0x7b, 0xd4, 0xcf, 0x27, 0x96, 0xef, 0xb2, + 0x89, 0xa1, 0x0e, 0x48, 0x49, 0xec, 0xde, 0xf7, 0x3f, 0x72, 0xff, 0x2d, 0x41, 0x54, 0x8c, 0x08, + 0x75, 0x01, 0xc4, 0xf4, 0x98, 0x06, 0x37, 0xf5, 0x54, 0xf9, 0xee, 0x51, 0x3f, 0x1f, 0xaf, 0x30, + 0x69, 0xbd, 0x4a, 0x9e, 0xf6, 0xf3, 0x1f, 0x3e, 0xab, 0xfb, 0xf0, 0x40, 0x94, 0x38, 0x6f, 0xa4, + 0x6e, 0xb0, 0x34, 0xeb, 0xbe, 0x46, 0xd4, 0x7d, 0x93, 0xb8, 0xf6, 0x9e, 0xa3, 0x75, 0xc4, 0xf5, + 0x6f, 0x72, 0x5f, 0x23, 0xab, 0x9e, 0x0c, 0xe5, 0x68, 0x20, 0xf6, 0x80, 0x3f, 0x7a, 0xe1, 0xec, + 0xe2, 0x7f, 0xa3, 0x25, 0xb8, 0xe4, 0x57, 0x56, 0xe9, 0x4c, 0xb5, 0x7a, 0xfa, 0x01, 0x66, 0x0e, + 0x87, 0x32, 0xf9, 0x9c, 0x5f, 0xb8, 0xae, 0x3d, 0x2a, 0xf3, 0xa2, 0xc2, 0x25, 0x98, 0x0b, 0x2c, + 0xab, 0x1f, 0x36, 0x63, 0x90, 0xf9, 0x2d, 0x75, 0xe0, 0xc5, 0xec, 0x5d, 0x98, 0x1d, 0x79, 0x91, + 0x2e, 0xc8, 0x36, 0x98, 0x1e, 0x1c, 0x7e, 0xc2, 0x5e, 0xac, 0xf0, 0x4f, 0x2f, 0x74, 0x4f, 0xeb, + 0x43, 0xdf, 0x85, 0x39, 0xc8, 0xf8, 0xcd, 0xf8, 0x6d, 0xff, 0x26, 0x09, 0xd1, 0x2d, 0xed, 0xb0, + 0x6d, 0x6b, 0x06, 0x5a, 0x80, 0x84, 0xf7, 0x5a, 0xc5, 0x6b, 0x2f, 0xae, 0x04, 0x45, 0xc3, 0x66, + 0x26, 0x8f, 0x9a, 0x99, 0x09, 0xe9, 0x1e, 0xc1, 0x0e, 0xb5, 0x00, 0x95, 0x3d, 0x9f, 0xe7, 0xbe, + 0xa5, 0x5c, 0x7e, 0xda, 0xcf, 0xdf, 0x9e, 0x6c, 0xf1, 0xb0, 0xde, 0x73, 0x4c, 0xf7, 0xb0, 0xd8, + 0xbc, 0x7b, 0x67, 0x47, 0x40, 0xd1, 0x2d, 0x6e, 0x2b, 0xa9, 0x5e, 0xf0, 0x53, 0x3c, 0x4a, 0xa2, + 0x0b, 0xa1, 0x76, 0x4c, 0xdd, 0xb1, 0x89, 0x77, 0x1b, 0x22, 0xa4, 0xeb, 0x4c, 0x88, 0x5e, 0x85, + 0xd9, 0x5d, 0xd3, 0x62, 0x17, 0x7a, 0x9e, 0x1e, 0xbf, 0x08, 0x49, 0x7b, 0x62, 0xa1, 0xf8, 0x00, + 0xd2, 0x81, 0xd7, 0x40, 0xd4, 0x08, 0x23, 0xcc, 0x08, 0x37, 0x8f, 0xfa, 0xf9, 0xd4, 0x60, 0x53, + 0x73, 0x43, 0xbc, 0x48, 0x1c, 0x93, 0x1a, 0x34, 0x43, 0xcd, 0x70, 0x1e, 0x66, 0xd8, 0x1f, 0x57, + 0xf0, 0xe7, 0x94, 0x0a, 0xff, 0x40, 0x35, 0x48, 0x89, 0x4c, 0x05, 0xff, 0xcb, 0x0b, 0xf1, 0x84, + 0x6a, 0x21, 0xb0, 0xf4, 0xde, 0xdf, 0x66, 0x14, 0x6b, 0x96, 0x6e, 0x1b, 0xd8, 0xa8, 0xd1, 0x6f, + 0x45, 0x24, 0x66, 0xd9, 0x07, 0x41, 0x2b, 0x90, 0xd6, 0xdb, 0x58, 0xb3, 0x7a, 0x5d, 0x0f, 0x07, + 0x4d, 0x88, 0x93, 0x12, 0xf5, 0x04, 0xd0, 0x06, 0xa0, 0x5d, 0xf6, 0xa6, 0x25, 0xd8, 0x2b, 0x76, + 0xf1, 0x37, 0x09, 0x98, 0xcc, 0xea, 0x2a, 0x83, 0x9e, 0xa1, 0xb7, 0x61, 0xa6, 0x8d, 0x35, 0x82, + 0xc5, 0x5b, 0xd9, 0x85, 0x53, 0x68, 0x90, 0xfd, 0xf5, 0x85, 0xc2, 0xd5, 0xd1, 0xcb, 0x90, 0xb2, + 0x6c, 0x4b, 0xd7, 0x2c, 0x1d, 0xb7, 0x19, 0x23, 0xf3, 0x3b, 0xc6, 0x61, 0x21, 0x2a, 0x43, 0x84, + 0xdf, 0x46, 0xb3, 0xbb, 0xf4, 0xf1, 0x84, 0xfa, 0x89, 0x8f, 0x95, 0x57, 0xa7, 0x14, 0x51, 0x13, + 0xd5, 0x20, 0xea, 0xf0, 0x97, 0x11, 0xec, 0x86, 0xfa, 0xcc, 0x14, 0x51, 0xe0, 0x75, 0xca, 0xea, + 0x94, 0xe2, 0xd5, 0x45, 0xdb, 0xde, 0x43, 0x41, 0xee, 0x75, 0xc5, 0x5b, 0xaf, 0xe2, 0x84, 0xc7, + 0x88, 0x01, 0xe0, 0x10, 0x0a, 0x1d, 0xa0, 0xc9, 0x6e, 0x98, 0xd8, 0xdd, 0xf5, 0xe9, 0x03, 0x1c, + 0x7a, 0x23, 0x41, 0x07, 0xc8, 0x6b, 0xa2, 0x0d, 0x00, 0xdd, 0x8f, 0x04, 0xd8, 0xad, 0x76, 0x62, + 0xe9, 0xf5, 0xf3, 0x04, 0xc9, 0xab, 0x53, 0x4a, 0x00, 0x01, 0xdd, 0x85, 0x84, 0x3e, 0xa0, 0xb6, + 0xec, 0x2c, 0x03, 0x7c, 0xe3, 0x5c, 0xfe, 0x6d, 0x95, 0xfa, 0xb4, 0x81, 0x14, 0x7d, 0x0a, 0x69, + 0x32, 0x74, 0xa8, 0xca, 0x5e, 0x62, 0xa8, 0x6f, 0x9e, 0x37, 0x0d, 0xbb, 0x3a, 0xa5, 0x8c, 0x20, + 0xa1, 0xdf, 0x05, 0xd9, 0x1d, 0xb9, 0xab, 0x61, 0xd7, 0xd2, 0xa7, 0x3f, 0xc4, 0x3b, 0xe1, 0x46, + 0x6a, 0x75, 0x4a, 0x19, 0x43, 0x43, 0x9f, 0xc1, 0x2c, 0x19, 0xfe, 0xa3, 0x82, 0xec, 0x15, 0xd6, + 0xc0, 0x37, 0x4f, 0xbd, 0x70, 0x38, 0xee, 0xcf, 0x27, 0x56, 0xa7, 0x94, 0x51, 0x2c, 0x0a, 0x6f, + 0x0d, 0x5f, 0xf9, 0xb0, 0x47, 0x0f, 0xa7, 0xc3, 0x1f, 0x7f, 0x05, 0x45, 0xe1, 0x47, 0xb0, 0xd0, + 0x1a, 0xc4, 0x3b, 0x9e, 0xaf, 0xc8, 0xbe, 0x70, 0xe6, 0x39, 0x64, 0xd4, 0x7d, 0xad, 0x4e, 0x29, + 0x83, 0xfa, 0xe5, 0x38, 0x44, 0xc5, 0xd5, 0xa0, 0x7f, 0x6f, 0x1f, 0x95, 0x63, 0x85, 0xdf, 0xc4, + 0x20, 0xe6, 0xc7, 0xa0, 0x8b, 0x80, 0xfc, 0x50, 0x63, 0xf0, 0x26, 0x95, 0xba, 0xa0, 0xd0, 0xea, + 0x94, 0x92, 0xf1, 0xca, 0x06, 0xcf, 0x52, 0x6f, 0x0f, 0x3d, 0x54, 0x9a, 0xe4, 0x0f, 0x3f, 0x68, + 0xa7, 0xfc, 0x97, 0x4c, 0xd4, 0x35, 0x74, 0x6c, 0xc3, 0xdc, 0x35, 0x07, 0xae, 0x81, 0x27, 0x92, + 0xd3, 0x9e, 0x58, 0xb8, 0x86, 0x6b, 0x90, 0x76, 0x7a, 0x16, 0xbb, 0x05, 0x14, 0xc9, 0x00, 0x1e, + 0x5f, 0xa5, 0x84, 0x54, 0x9c, 0xe7, 0x2b, 0x23, 0xac, 0x73, 0xe3, 0x4c, 0xd6, 0xf1, 0xc6, 0xbe, + 0x2a, 0xf9, 0xb4, 0xb3, 0x3c, 0x4a, 0x3b, 0x37, 0xcf, 0xa6, 0x9d, 0x00, 0x8c, 0xcf, 0x3b, 0x3b, + 0xc7, 0xf2, 0xce, 0xe2, 0x84, 0x1b, 0x27, 0x80, 0x38, 0x4c, 0x3c, 0x95, 0x11, 0xe2, 0xb9, 0x71, + 0x26, 0xf1, 0x04, 0xc7, 0x28, 0x98, 0x67, 0xf3, 0x18, 0xe6, 0x79, 0x63, 0x22, 0xe6, 0x09, 0x80, + 0x05, 0xa9, 0x47, 0x39, 0x8e, 0x7a, 0x8a, 0x93, 0x51, 0x4f, 0x00, 0x72, 0x88, 0x7b, 0xbe, 0x37, + 0xc6, 0x3d, 0xf2, 0xd9, 0x9b, 0xf7, 0xd8, 0x0c, 0xd0, 0xaa, 0x34, 0x46, 0x3e, 0xda, 0x31, 0xe4, + 0x93, 0x61, 0xf0, 0x6f, 0x9d, 0x83, 0x7c, 0x02, 0x0d, 0x8c, 0xb3, 0xcf, 0xc7, 0x90, 0x0c, 0x32, + 0x06, 0x7b, 0xbb, 0x73, 0x3a, 0xb7, 0x9d, 0xf0, 0x17, 0x50, 0xcc, 0x06, 0x02, 0x45, 0xe8, 0xfb, + 0xe3, 0xc4, 0x33, 0x77, 0x26, 0xf8, 0x09, 0xb7, 0xd3, 0xab, 0xd2, 0x38, 0xf3, 0xdc, 0x09, 0x32, + 0xcf, 0xfc, 0x99, 0x7e, 0x69, 0x2c, 0xa2, 0x5d, 0x95, 0x82, 0xd4, 0x03, 0x10, 0xf3, 0x1e, 0x3f, + 0x04, 0x68, 0xa8, 0xf0, 0x53, 0x09, 0xa6, 0x1b, 0x76, 0x0b, 0xbd, 0x14, 0x48, 0x4d, 0xa6, 0xc4, + 0x51, 0x74, 0xa6, 0x61, 0xb7, 0x44, 0x8e, 0xf1, 0x83, 0x41, 0x6d, 0x71, 0xf8, 0xfb, 0xc6, 0x29, + 0x5d, 0xf1, 0x33, 0xbb, 0x7e, 0x25, 0xf4, 0x1e, 0x44, 0xbb, 0x3c, 0xb8, 0x16, 0x0c, 0x55, 0x38, + 0xad, 0x3e, 0xd7, 0x54, 0xbc, 0x2a, 0x37, 0x6f, 0x04, 0xff, 0xa4, 0x71, 0xdd, 0x36, 0x30, 0x4a, + 0x03, 0x6c, 0x69, 0x84, 0x74, 0xf7, 0x1d, 0x8d, 0x60, 0x79, 0x0a, 0x45, 0x61, 0x7a, 0x6d, 0xbd, + 0x29, 0x4b, 0x37, 0x3f, 0x0e, 0xe6, 0x15, 0xab, 0x4a, 0xa9, 0xbe, 0x51, 0xdf, 0x58, 0x51, 0x37, + 0x4a, 0xeb, 0xb5, 0xa6, 0x3c, 0x85, 0xb2, 0x30, 0xff, 0x51, 0xa9, 0xbe, 0x2d, 0x12, 0x8d, 0x6a, + 0x7d, 0x63, 0xbb, 0xa6, 0xdc, 0x2b, 0xdd, 0x91, 0x25, 0x74, 0x19, 0x90, 0xb2, 0x59, 0x59, 0x6b, + 0x56, 0xcb, 0x6a, 0x65, 0x73, 0x7d, 0xab, 0x54, 0xd9, 0xae, 0x6f, 0x6e, 0xc8, 0x21, 0x14, 0x83, + 0x70, 0x75, 0x73, 0xa3, 0x26, 0xc3, 0xcd, 0x9f, 0x84, 0x21, 0xcc, 0x52, 0x1a, 0x2f, 0x43, 0x62, + 0x67, 0xa3, 0xb9, 0x55, 0xab, 0xd4, 0x97, 0xeb, 0xb5, 0xaa, 0x3c, 0x95, 0x9b, 0x7b, 0xfc, 0x64, + 0x61, 0x96, 0x16, 0xed, 0x58, 0xa4, 0x8b, 0x75, 0xc6, 0x99, 0x28, 0x07, 0x91, 0x72, 0xa9, 0xb2, + 0xb6, 0xb3, 0x25, 0x4b, 0xb9, 0xf4, 0xe3, 0x27, 0x0b, 0x40, 0x15, 0x38, 0xdf, 0xa1, 0x17, 0x79, + 0xca, 0x63, 0x53, 0xa9, 0xc9, 0xa1, 0xdc, 0xec, 0xe3, 0x27, 0x0b, 0x09, 0x96, 0x49, 0x11, 0x9c, + 0xf5, 0x2a, 0xa4, 0x9a, 0x95, 0xd5, 0xda, 0x7a, 0x49, 0xad, 0xac, 0x96, 0x36, 0x56, 0x6a, 0xf2, + 0x74, 0x6e, 0xfe, 0xf1, 0x93, 0x05, 0x79, 0xd4, 0xee, 0x69, 0x13, 0xf5, 0xf5, 0xad, 0x4d, 0x65, + 0x5b, 0x0e, 0x0f, 0x9a, 0xe0, 0x74, 0x83, 0x0a, 0x00, 0xbc, 0xf6, 0x72, 0xad, 0x56, 0x95, 0x67, + 0x72, 0xe8, 0xf1, 0x93, 0x85, 0x34, 0x2d, 0x1f, 0xb0, 0x08, 0xba, 0x06, 0xc9, 0x8a, 0x52, 0x2b, + 0x6d, 0xd7, 0xd4, 0xe6, 0x76, 0x69, 0xbb, 0x29, 0x47, 0x06, 0x23, 0x09, 0x30, 0x03, 0x2a, 0x42, + 0xa6, 0xb4, 0xb3, 0xbd, 0xa9, 0x0e, 0xe9, 0x46, 0x73, 0x57, 0x1e, 0x3f, 0x59, 0x98, 0xa3, 0xba, + 0xa5, 0x9e, 0x6b, 0x07, 0xf5, 0x5f, 0x07, 0x79, 0xa8, 0xff, 0xea, 0x4a, 0x45, 0x8e, 0xe5, 0x2e, + 0x3f, 0x7e, 0xb2, 0x80, 0x46, 0x87, 0xb0, 0x52, 0x41, 0xdf, 0x82, 0xcb, 0xdb, 0x9f, 0x6c, 0xd5, + 0xaa, 0xb5, 0x66, 0x45, 0x1d, 0x1e, 0x76, 0x3c, 0x97, 0x7d, 0xfc, 0x64, 0x61, 0x9e, 0xd6, 0x19, + 0x1b, 0xfa, 0x1b, 0x20, 0x37, 0xb7, 0x95, 0x5a, 0x69, 0x5d, 0xad, 0x6f, 0xac, 0xd4, 0x9a, 0x6c, + 0xb1, 0x60, 0xd0, 0xa5, 0x91, 0x3d, 0x4c, 0x87, 0xb0, 0x51, 0xfb, 0x68, 0x04, 0x3f, 0x31, 0xd0, + 0x1f, 0xd9, 0x96, 0x68, 0x01, 0xe2, 0xeb, 0xf5, 0x15, 0xa5, 0xc4, 0x70, 0x93, 0xb9, 0xcc, 0xe3, + 0x27, 0x0b, 0x29, 0xaa, 0xe7, 0x6f, 0xb2, 0x5c, 0xec, 0xc7, 0x3f, 0xbd, 0x3a, 0xf5, 0xd7, 0x3f, + 0xbb, 0x3a, 0x55, 0xbe, 0xfe, 0xf9, 0x7f, 0x5e, 0x9d, 0xfa, 0xfc, 0xe8, 0xaa, 0xf4, 0xab, 0xa3, + 0xab, 0xd2, 0x17, 0x47, 0x57, 0xa5, 0xff, 0x38, 0xba, 0x2a, 0xfd, 0xc9, 0x97, 0x57, 0xa7, 0x7e, + 0xf5, 0xe5, 0xd5, 0xa9, 0x2f, 0xbe, 0xbc, 0x3a, 0xf5, 0x69, 0x84, 0xdb, 0x75, 0x2b, 0xc2, 0xce, + 0x82, 0x6f, 0xfd, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x06, 0x36, 0x30, 0x9c, 0x83, 0x3e, 0x00, + 0x00, } func (this *Lease) Equal(that interface{}) bool { @@ -3506,14 +3508,14 @@ func (m *RestoreDetails) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x92 } } - if len(m.SystemTablesRestored) > 0 { - keysForSystemTablesRestored := make([]string, 0, len(m.SystemTablesRestored)) - for k := range m.SystemTablesRestored { - keysForSystemTablesRestored = append(keysForSystemTablesRestored, string(k)) + if len(m.SystemTablesMigrated) > 0 { + keysForSystemTablesMigrated := make([]string, 0, len(m.SystemTablesMigrated)) + for k := range m.SystemTablesMigrated { + keysForSystemTablesMigrated = append(keysForSystemTablesMigrated, string(k)) } - github_com_gogo_protobuf_sortkeys.Strings(keysForSystemTablesRestored) - for iNdEx := len(keysForSystemTablesRestored) - 1; iNdEx >= 0; iNdEx-- { - v := m.SystemTablesRestored[string(keysForSystemTablesRestored[iNdEx])] + github_com_gogo_protobuf_sortkeys.Strings(keysForSystemTablesMigrated) + for iNdEx := len(keysForSystemTablesMigrated) - 1; iNdEx >= 0; iNdEx-- { + v := m.SystemTablesMigrated[string(keysForSystemTablesMigrated[iNdEx])] baseI := i i-- if v { @@ -3523,9 +3525,9 @@ func (m *RestoreDetails) MarshalToSizedBuffer(dAtA []byte) (int, error) { } i-- dAtA[i] = 0x10 - i -= len(keysForSystemTablesRestored[iNdEx]) - copy(dAtA[i:], keysForSystemTablesRestored[iNdEx]) - i = encodeVarintJobs(dAtA, i, uint64(len(keysForSystemTablesRestored[iNdEx]))) + i -= len(keysForSystemTablesMigrated[iNdEx]) + copy(dAtA[i:], keysForSystemTablesMigrated[iNdEx]) + i = encodeVarintJobs(dAtA, i, uint64(len(keysForSystemTablesMigrated[iNdEx]))) i-- dAtA[i] = 0xa i = encodeVarintJobs(dAtA, i, uint64(baseI-i)) @@ -6644,8 +6646,8 @@ func (m *RestoreDetails) Size() (n int) { n += 2 + l + sovJobs(uint64(l)) } } - if len(m.SystemTablesRestored) > 0 { - for k, v := range m.SystemTablesRestored { + if len(m.SystemTablesMigrated) > 0 { + for k, v := range m.SystemTablesMigrated { _ = k _ = v mapEntrySize := 1 + len(k) + sovJobs(uint64(len(k))) + 1 + 1 @@ -9685,7 +9687,7 @@ func (m *RestoreDetails) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 17: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SystemTablesRestored", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SystemTablesMigrated", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -9712,8 +9714,8 @@ func (m *RestoreDetails) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.SystemTablesRestored == nil { - m.SystemTablesRestored = make(map[string]bool) + if m.SystemTablesMigrated == nil { + m.SystemTablesMigrated = make(map[string]bool) } var mapkey string var mapvalue bool @@ -9796,7 +9798,7 @@ func (m *RestoreDetails) Unmarshal(dAtA []byte) error { iNdEx += skippy } } - m.SystemTablesRestored[mapkey] = mapvalue + m.SystemTablesMigrated[mapkey] = mapvalue iNdEx = postIndex case 18: if wireType != 2 { diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index e46fed79b07b..459c913e6668 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -184,12 +184,13 @@ message RestoreDetails { // stages have completed via these flags. bool prepare_completed = 8; bool stats_inserted = 9; - // SystemTablesRestored keeps track of dynamic states that need to happen only - // once during the lifetime of a job. Note, that this state may be shared - // between job versions, so updates to this map must be considered carefully. - // It maps system table names to whether or not they have already been - // restored. - map system_tables_restored = 17; + // SystemTablesMigrated keeps track of which system tables data have been + // migrated. We need to keep track of this because if we've modified the + // restored data via a migration, we can't restore back into that span as the + // migrated keys will shadow the ones that will be restored. + // Note, that this state may be shared between job versions, so updates to + // this map must be considered carefully. + map system_tables_migrated = 17; // DescriptorsPublished indicates whether or not the descriptors written in // the job have been transactionally updated after the data was restored. bool descriptors_published = 10; diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_info.go b/pkg/kv/kvserver/batcheval/cmd_lease_info.go index 1cdcb9238645..722a227b8c1a 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_info.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_info.go @@ -40,8 +40,10 @@ func LeaseInfo( // If there's a lease request in progress, speculatively return that future // lease. reply.Lease = nextLease + reply.CurrentLease = &lease } else { reply.Lease = lease } + reply.EvaluatedBy = cArgs.EvalCtx.StoreID() return result.Result{}, nil } diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index 06b593b361c4..9e0e2dffb00f 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -1424,10 +1424,11 @@ func TestLeaseNotUsedAfterRestart(t *testing.T) { // Test that a lease extension (a RequestLeaseRequest that doesn't change the // lease holder) is not blocked by ongoing reads. The test relies on the fact // that RequestLeaseRequest does not declare to touch the whole key span of the -// range, and thus don't conflict through the command queue with other reads. +// range, and thus don't conflict through latches with other reads. func TestLeaseExtensionNotBlockedByRead(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + ctx := context.Background() readBlocked := make(chan struct{}) cmdFilter := func(fArgs kvserverbase.FilterArgs) *roachpb.Error { if fArgs.Hdr.UserPriority == 42 { @@ -1449,7 +1450,7 @@ func TestLeaseExtensionNotBlockedByRead(t *testing.T) { }, }) s := srv.(*server.TestServer) - defer s.Stopper().Stop(context.Background()) + defer s.Stopper().Stop(ctx) store, err := s.GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID()) if err != nil { @@ -1465,7 +1466,7 @@ func TestLeaseExtensionNotBlockedByRead(t *testing.T) { Key: key, }, } - if _, pErr := kv.SendWrappedWith(context.Background(), s.DB().NonTransactionalSender(), + if _, pErr := kv.SendWrappedWith(ctx, s.DB().NonTransactionalSender(), roachpb.Header{UserPriority: 42}, &getReq); pErr != nil { errChan <- pErr.GoError() @@ -1502,21 +1503,21 @@ func TestLeaseExtensionNotBlockedByRead(t *testing.T) { } for { - curLease, _, err := s.GetRangeLease(context.Background(), key) + leaseInfo, _, err := s.GetRangeLease(ctx, key, server.AllowQueryToBeForwardedToDifferentNode) if err != nil { t.Fatal(err) } - leaseReq.PrevLease = curLease + leaseReq.PrevLease = leaseInfo.CurrentOrProspective() - _, pErr := kv.SendWrapped(context.Background(), s.DB().NonTransactionalSender(), &leaseReq) + _, pErr := kv.SendWrapped(ctx, s.DB().NonTransactionalSender(), &leaseReq) if _, ok := pErr.GetDetail().(*roachpb.AmbiguousResultError); ok { - log.Infof(context.Background(), "retrying lease after %s", pErr) + log.Infof(ctx, "retrying lease after %s", pErr) continue } if _, ok := pErr.GetDetail().(*roachpb.LeaseRejectedError); ok { // Lease rejected? Try again. The extension should work because // extending is idempotent (assuming the PrevLease matches). - log.Infof(context.Background(), "retrying lease after %s", pErr) + log.Infof(ctx, "retrying lease after %s", pErr) continue } if pErr != nil { diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index a32673dc1a39..eeea580b89a5 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -42,6 +42,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" + "github.com/cockroachdb/logtags" "go.etcd.io/etcd/raft/v3" ) @@ -546,3 +547,11 @@ func WatchForDisappearingReplicas(t testing.TB, store *Store) { } } } + +// AcquireLease is redirectOnOrAcquireLease exposed for tests. +func (r *Replica) AcquireLease(ctx context.Context) (kvserverpb.LeaseStatus, error) { + ctx = r.AnnotateCtx(ctx) + ctx = logtags.AddTag(ctx, "lease-acq", nil) + l, pErr := r.redirectOnOrAcquireLease(ctx) + return l, pErr.GoError() +} diff --git a/pkg/kv/kvserver/replica_closedts_test.go b/pkg/kv/kvserver/replica_closedts_test.go index 87a0b7d95638..a68484da4668 100644 --- a/pkg/kv/kvserver/replica_closedts_test.go +++ b/pkg/kv/kvserver/replica_closedts_test.go @@ -13,6 +13,7 @@ package kvserver_test import ( "context" "sync" + "sync/atomic" "testing" "github.com/cockroachdb/cockroach/pkg/base" @@ -33,9 +34,9 @@ import ( ) // TestBumpSideTransportClosed tests the various states that a replica can find -// itself in when its TestBumpSideTransportClosed is called. It verifies that -// the method only returns successfully if it can bump its closed timestamp to -// the target. +// itself in when its BumpSideTransportClosed is called. It verifies that the +// method only returns successfully if it can bump its closed timestamp to the +// target. func TestBumpSideTransportClosed(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -413,6 +414,242 @@ func TestBumpSideTransportClosed(t *testing.T) { } } +// Test that a lease proposal that gets rejected doesn't erroneously dictate the +// closed timestamp of further requests. If it would, then writes could violate +// that closed timestamp. The tricky scenario tested is the following: +// +// 1. A lease held by rep1 is getting close to its expiration. +// 2. Rep1 begins the process of transferring its lease to rep2 with a start +// time of 100. +// 3. The transfer goes slowly. From the perspective of rep2, the original lease +// expires, so it begins acquiring a new lease with a start time of 200. The +// lease acquisition is slow to propose. +// 4. The lease transfer finally applies. Rep2 is the new leaseholder and bumps +// its tscache to 100. +// 5. Two writes start evaluating on rep2 under the new lease. They bump their +// write timestamp to 100,1. +// 6. Rep2's lease acquisition from step 3 is proposed. Here's where the +// regression that this test is protecting against comes in: if rep2 was to +// mechanically bump its assignedClosedTimestamp to 200, that'd be incorrect +// because there are in-flight writes at 100. If those writes get proposed +// after the lease acquisition request, the second of them to get proposed +// would violate the closed time carried by the first (see below). +// 7. The lease acquisition gets rejected below Raft because the previous lease +// it asserts doesn't correspond to the lease that it applies under. +// 8. The two writes from step 5 are proposed. The closed timestamp that they +// each carry has a lower bound of rep2.assignedClosedTimestmap. If this was +// 200, then the second one would violate the closed timestamp carried by the +// first one - the first one says that 200 is closed, but then the second +// tries to write at 100. Note that the first write is OK writing at 100 even +// though it carries a closed timestamp of 200 - the closed timestamp carried +// by a command only binds future commands. +// +// The test simulates the scenario and verifies that we don't crash with a +// closed timestamp violation assertion. We avoid the violation because, in step +// 6, the lease proposal doesn't bump the assignedClosedTimestamp. +func TestRejectedLeaseDoesntDictateClosedTimestamp(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + ctx := context.Background() + + // We're going to orchestrate the scenario by controlling the timing of the + // lease transfer, the lease acquisition and the writes. Note that we'll block + // the lease acquisition and the writes after they evaluate but before they + // get proposed, but we'll block the lease transfer when it's just about to be + // proposed, after it gets assigned the closed timestamp that it will carry. + // We want it to carry a relatively low closed timestamp, so we want its + // closed timestamp to be assigned before we bump the clock to expire the + // original lease. + + // leaseTransferCh is used to block the lease transfer. + leaseTransferCh := make(chan struct{}) + // leaseAcqCh is used to block the lease acquisition. + leaseAcqCh := make(chan struct{}) + // writeCh is used to wait for the two writes to block. + writeCh := make(chan struct{}) + // unblockWritesCh is used to unblock the two writes. + unblockWritesCh := make(chan struct{}) + var writeKey1, writeKey2 atomic.Value + // Initialize the atomics so they get bound to a specific type. + writeKey1.Store(roachpb.Key{}) + writeKey2.Store(roachpb.Key{}) + var blockedRangeID int64 + var trappedLeaseAcquisition int64 + + blockLeaseAcquisition := func(args kvserverbase.FilterArgs) { + blockedRID := roachpb.RangeID(atomic.LoadInt64(&blockedRangeID)) + leaseReq, ok := args.Req.(*roachpb.RequestLeaseRequest) + if !ok || args.Hdr.RangeID != blockedRID || leaseReq.Lease.Replica.NodeID != 2 { + return + } + if atomic.CompareAndSwapInt64(&trappedLeaseAcquisition, 0, 1) { + leaseAcqCh <- struct{}{} + <-leaseAcqCh + } + } + + blockWrites := func(args kvserverbase.FilterArgs) { + wk1 := writeKey1.Load().(roachpb.Key) + wk2 := writeKey2.Load().(roachpb.Key) + if put, ok := args.Req.(*roachpb.PutRequest); ok && (put.Key.Equal(wk1) || put.Key.Equal(wk2)) { + writeCh <- struct{}{} + <-unblockWritesCh + } + } + + blockTransfer := func(p *kvserver.ProposalData) { + blockedRID := roachpb.RangeID(atomic.LoadInt64(&blockedRangeID)) + ba := p.Request + if ba.RangeID != blockedRID { + return + } + _, ok := p.Request.GetArg(roachpb.TransferLease) + if !ok { + return + } + leaseTransferCh <- struct{}{} + <-leaseTransferCh + } + + manual := hlc.NewHybridManualClock() + tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manual.UnixNano, + }, + Store: &kvserver.StoreTestingKnobs{ + DisableConsistencyQueue: true, + EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ + TestingPostEvalFilter: func(args kvserverbase.FilterArgs) *roachpb.Error { + blockWrites(args) + blockLeaseAcquisition(args) + return nil + }, + }, + TestingProposalSubmitFilter: func(p *kvserver.ProposalData) (drop bool, _ error) { + blockTransfer(p) + return false, nil + }, + }, + }, + }}) + defer tc.Stopper().Stop(ctx) + + manual.Pause() + // Upreplicate a range. + n1, n2 := tc.Servers[0], tc.Servers[1] + // One of the filters hardcodes a node id. + require.Equal(t, roachpb.NodeID(2), n2.NodeID()) + key := tc.ScratchRangeWithExpirationLease(t) + s1 := tc.GetFirstStoreFromServer(t, 0) + t1, t2 := tc.Target(0), tc.Target(1) + repl0 := s1.LookupReplica(keys.MustAddr(key)) + desc := *repl0.Desc() + require.NotNil(t, repl0) + tc.AddVotersOrFatal(t, key, t2) + require.NoError(t, tc.WaitForVoters(key, t2)) + // Make sure the lease starts off on n1. + lease, _ /* now */, err := tc.FindRangeLease(desc, &t1 /* hint */) + require.NoError(t, err) + require.Equal(t, n1.NodeID(), lease.Replica.NodeID) + + // Advance the time a bit. We'll then initiate a transfer, and we want the + // transferred lease to be valid for a while after the original lease expires. + remainingNanos := lease.GetExpiration().WallTime - manual.UnixNano() + // NOTE: We don't advance the clock past the mid-point of the lease, otherwise + // it gets extended. + pause1 := remainingNanos / 3 + manual.Increment(pause1) + + // Start a lease transfer from n1 to n2. We'll block the proposal of the transfer for a while. + atomic.StoreInt64(&blockedRangeID, int64(desc.RangeID)) + transferErrCh := make(chan error) + go func() { + transferErrCh <- tc.TransferRangeLease(desc, t2) + }() + defer func() { + require.NoError(t, <-transferErrCh) + }() + // Wait for the lease transfer to evaluate and then block. + <-leaseTransferCh + // With the lease transfer still blocked, we now advance the clock beyond the + // original lease's expiration and we make n2 try to acquire a lease. This + // lease acquisition request will also be blocked. + manual.Increment(remainingNanos - pause1 + 1) + leaseAcqErrCh := make(chan error) + go func() { + r, _, err := n2.Stores().GetReplicaForRangeID(ctx, desc.RangeID) + if err != nil { + leaseAcqErrCh <- err + return + } + _, err = r.AcquireLease(ctx) + leaseAcqErrCh <- err + }() + // Wait for the lease acquisition to be blocked. + select { + case <-leaseAcqCh: + case err := <-leaseAcqErrCh: + t.Fatalf("lease request unexpectedly finished. err: %v", err) + } + // Let the previously blocked transfer succeed. n2's lease acquisition remains + // blocked. + close(leaseTransferCh) + // Wait until n2 has applied the lease transfer. + desc = *repl0.Desc() + testutils.SucceedsSoon(t, func() error { + li, _ /* now */, err := tc.FindRangeLeaseEx(ctx, desc, &t2 /* hint */) + if err != nil { + return err + } + lease = li.Current() + if !lease.OwnedBy(n2.GetFirstStoreID()) { + return errors.Errorf("n2 still unaware of its lease: %s", li.Current()) + } + return nil + }) + + // Now we send two writes. We'll block them after evaluation. Then we'll + // unblock the lease acquisition, let the respective command fail to apply, + // and then we'll unblock the writes. + err1 := make(chan error) + err2 := make(chan error) + go func() { + writeKey1.Store(key) + sender := n2.DB().NonTransactionalSender() + pArgs := putArgs(key, []byte("test val")) + _, pErr := kv.SendWrappedWith(ctx, sender, roachpb.Header{Timestamp: lease.Start.ToTimestamp()}, pArgs) + err1 <- pErr.GoError() + }() + go func() { + k := key.Next() + writeKey2.Store(k) + sender := n2.DB().NonTransactionalSender() + pArgs := putArgs(k, []byte("test val2")) + _, pErr := kv.SendWrappedWith(ctx, sender, roachpb.Header{Timestamp: lease.Start.ToTimestamp()}, pArgs) + err2 <- pErr.GoError() + }() + // Wait for the writes to evaluate and block before proposal. + <-writeCh + <-writeCh + + // Unblock the lease acquisition. + close(leaseAcqCh) + if err := <-leaseAcqErrCh; err != nil { + close(unblockWritesCh) + t.Fatal(err) + } + + // Now unblock the writes. + close(unblockWritesCh) + require.NoError(t, <-err1) + require.NoError(t, <-err2) + // Not crashing with a closed timestamp violation assertion marks the success + // of this test. +} + // BenchmarkBumpSideTransportClosed measures the latency of a single call to // (*Replica).BumpSideTransportClosed. The closed timestamp side-transport was // designed with a performance expectation of this check taking no more than diff --git a/pkg/kv/kvserver/replica_init.go b/pkg/kv/kvserver/replica_init.go index c64027597c26..203762749be1 100644 --- a/pkg/kv/kvserver/replica_init.go +++ b/pkg/kv/kvserver/replica_init.go @@ -99,6 +99,7 @@ func newUnloadedReplica( r.mu.proposalBuf.Init((*replicaProposer)(r), tracker.NewLockfreeTracker(), r.Clock(), r.ClusterSettings()) r.mu.proposalBuf.testing.allowLeaseProposalWhenNotLeader = store.cfg.TestingKnobs.AllowLeaseRequestProposalsWhenNotLeader r.mu.proposalBuf.testing.dontCloseTimestamps = store.cfg.TestingKnobs.DontCloseTimestamps + r.mu.proposalBuf.testing.submitProposalFilter = store.cfg.TestingKnobs.TestingProposalSubmitFilter if leaseHistoryMaxEntries > 0 { r.leaseHistory = newLeaseHistory() diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index 85deb72a6e99..cccb15f2ada6 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -150,11 +150,12 @@ type propBuf struct { cnt propBufCnt arr propBufArray - // assignedClosedTimestamp is the largest "closed timestamp" - i.e. the largest - // timestamp that was communicated to other replicas as closed, representing a - // promise that this leaseholder will not evaluate writes below this timestamp - // any more. It is set when proposals are flushed from the buffer, and also - // by the side-transport which closes timestamps out of band. + // assignedClosedTimestamp is the largest "closed timestamp" - i.e. the + // largest timestamp that was communicated to other replicas as closed, + // representing a promise that this leaseholder will not evaluate writes with + // timestamp <= assignedClosedTimestamp any more. It is set when proposals are + // flushed from the buffer, and also by the side-transport which closes + // timestamps out of band. // // Note that this field is not used by the local replica (or by anybody) // directly to decide whether follower reads can be served. See @@ -594,17 +595,6 @@ func (b *propBuf) FlushLockedWithRaftGroup( // Exit the tracker. p.tok.doneIfNotMovedLocked(ctx) - // Potentially drop the proposal before passing it to etcd/raft, but - // only after performing necessary bookkeeping. - if filter := b.testing.submitProposalFilter; filter != nil { - if drop, err := filter(p); drop || err != nil { - if firstErr == nil { - firstErr = err - } - continue - } - } - // If we don't have a raft group or if the raft group has rejected one // of the proposals, we don't try to propose any more proposals. The // rest of the proposals will still be registered with the proposer, so @@ -627,6 +617,17 @@ func (b *propBuf) FlushLockedWithRaftGroup( } } + // Potentially drop the proposal before passing it to etcd/raft, but + // only after performing necessary bookkeeping. + if filter := b.testing.submitProposalFilter; filter != nil { + if drop, err := filter(p); drop || err != nil { + if firstErr == nil { + firstErr = err + } + continue + } + } + // Coordinate proposing the command to etcd/raft. if crt := p.command.ReplicatedEvalResult.ChangeReplicas; crt != nil { // Flush any previously batched (non-conf change) proposals to @@ -713,7 +714,6 @@ func (b *propBuf) assignClosedTimestampToProposalLocked( // Lease transfers behave like regular proposals. Note that transfers // carry a summary of the timestamp cache, so the new leaseholder will be // aware of all the reads performed by the previous leaseholder. - isBrandNewLeaseRequest := false if p.Request.IsLeaseRequest() { // We read the lease from the ReplicatedEvalResult, not from leaseReq, because the // former is more up to date, having been modified by the evaluation. @@ -723,12 +723,51 @@ func (b *propBuf) assignClosedTimestampToProposalLocked( if leaseExtension { return nil } - isBrandNewLeaseRequest = true // For brand new leases, we close the lease start time. Since this proposing // replica is not the leaseholder, the previous target is meaningless. closedTSTarget = newLease.Start.ToTimestamp() - } - if !isBrandNewLeaseRequest { + // We forward closedTSTarget to b.assignedClosedTimestamp. We surprisingly + // might have previously closed a timestamp above the lease start time - + // when we close timestamps in the future, then attempt to transfer our + // lease away (and thus proscribe it) but the transfer fails and we're now + // acquiring a new lease to replace the proscribed one. + // + // TODO(andrei,nvanbenschoten): add a test with scenario: + // - Acquire lease @ 1 + // - Close future timestamp @ 3 + // - Attempt to transfer lease @ 2 + // - Reject + // - Reacquire lease @ 2 + closedTSTarget.Forward(b.assignedClosedTimestamp) + + // Note that we're not bumping b.assignedClosedTimestamp here (we're not + // calling forwardClosedTimestampLocked). Bumping it to the lease start time + // would (surprisingly?) be illegal: just because we're proposing a lease + // starting at timestamp 100, doesn't mean we're sure to not be in the + // process of evaluating requests writing below 100. This can happen if a + // lease transfer has already applied while we were evaluating this lease + // request, and if we've already started evaluating writes under the + // transferred lease. Such a transfer can give us the lease starting at + // timestamp 50. If such a transfer applied, then our lease request that + // we're proposing now is sure to not apply. But if we were to bump + // b.assignedClosedTimestamp, the damage would be done. See + // TestRejectedLeaseDoesntDictateClosedTimestamp. + } else { + // Sanity check that this command is not violating the closed timestamp. It + // must be writing at a timestamp above assignedClosedTimestamp + // (assignedClosedTimestamp represents the promise that this replica made + // through previous commands to not evaluate requests with lower + // timestamps); in other words, assignedClosedTimestamp was not supposed to + // have been incremented while requests with lower timestamps were + // evaluating (instead, assignedClosedTimestamp was supposed to have bumped + // the write timestamp of any request the began evaluating after it was + // set). + if p.Request.WriteTimestamp().Less(b.assignedClosedTimestamp) && p.Request.IsIntentWrite() { + return errors.AssertionFailedf("attempting to propose command writing below closed timestamp. "+ + "wts: %s < assigned closed: %s; ba: %s", + p.Request.WriteTimestamp(), b.assignedClosedTimestamp, p.Request) + } + lb := b.evalTracker.LowerBound(ctx) if !lb.IsEmpty() { // If the tracker told us that requests are currently evaluating at @@ -739,15 +778,18 @@ func (b *propBuf) assignClosedTimestampToProposalLocked( } // We can't close timestamps above the current lease's expiration. closedTSTarget.Backward(p.leaseStatus.ClosedTimestampUpperBound()) + + // We're about to close closedTSTarget. The propBuf needs to remember that + // in order for incoming requests to be bumped above it (through + // TrackEvaluatingRequest). + if !b.forwardClosedTimestampLocked(closedTSTarget) { + closedTSTarget = b.assignedClosedTimestamp + } } - // We're about to close closedTSTarget. The propBuf needs to remember that in - // order for incoming requests to be bumped above it (through - // TrackEvaluatingRequest). - b.forwardClosedTimestampLocked(closedTSTarget) // Fill in the closed ts in the proposal. f := &b.tmpClosedTimestampFooter - f.ClosedTimestamp = b.assignedClosedTimestamp + f.ClosedTimestamp = closedTSTarget footerLen := f.Size() if log.ExpensiveLogEnabled(ctx, 4) { log.VEventf(ctx, 4, "attaching closed timestamp %s to proposal %x", b.assignedClosedTimestamp, p.idKey) diff --git a/pkg/kv/kvserver/replica_proposal_buf_test.go b/pkg/kv/kvserver/replica_proposal_buf_test.go index f7dd181a69d9..b032f44d9076 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -201,9 +201,10 @@ type proposalCreator struct { lease kvserverpb.LeaseStatus } -func (pc proposalCreator) newPutProposal() (*ProposalData, []byte) { +func (pc proposalCreator) newPutProposal(ts hlc.Timestamp) (*ProposalData, []byte) { var ba roachpb.BatchRequest ba.Add(&roachpb.PutRequest{}) + ba.Timestamp = ts return pc.newProposal(ba) } @@ -268,7 +269,7 @@ func TestProposalBuffer(t *testing.T) { if leaseReq { pd, data = pc.newLeaseProposal(roachpb.Lease{}) } else { - pd, data = pc.newPutProposal() + pd, data = pc.newPutProposal(hlc.Timestamp{}) } _, tok := b.TrackEvaluatingRequest(ctx, hlc.MinTimestamp) mlai, err := b.Insert(ctx, pd, data, tok) @@ -293,7 +294,7 @@ func TestProposalBuffer(t *testing.T) { // Insert another proposal. This causes the buffer to flush. Doing so // results in a lease applied index being skipped, which is harmless. // Remember that the lease request above did not receive a lease index. - pd, data := pc.newPutProposal() + pd, data := pc.newPutProposal(hlc.Timestamp{}) _, tok := b.TrackEvaluatingRequest(ctx, hlc.MinTimestamp) mlai, err := b.Insert(ctx, pd, data, tok) require.Nil(t, err) @@ -361,7 +362,7 @@ func TestProposalBufferConcurrentWithDestroy(t *testing.T) { for i := 0; i < concurrency; i++ { g.Go(func() error { for { - pd, data := pc.newPutProposal() + pd, data := pc.newPutProposal(hlc.Timestamp{}) _, tok := b.TrackEvaluatingRequest(ctx, hlc.MinTimestamp) mlai, err := b.Insert(ctx, pd, data, tok) if err != nil { @@ -420,7 +421,9 @@ func TestProposalBufferRegistersAllOnProposalError(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() + raft := &testProposerRaft{} var p testProposer + p.raftGroup = raft var b propBuf var pc proposalCreator clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) @@ -429,7 +432,7 @@ func TestProposalBufferRegistersAllOnProposalError(t *testing.T) { num := propBufArrayMinSize toks := make([]TrackedRequestToken, num) for i := 0; i < num; i++ { - pd, data := pc.newPutProposal() + pd, data := pc.newPutProposal(hlc.Timestamp{}) _, toks[i] = b.TrackEvaluatingRequest(ctx, hlc.MinTimestamp) _, err := b.Insert(ctx, pd, data, toks[i]) require.Nil(t, err) @@ -474,7 +477,7 @@ func TestProposalBufferRegistrationWithInsertionErrors(t *testing.T) { if i%2 == 0 { pd, data = pc.newLeaseProposal(roachpb.Lease{}) } else { - pd, data = pc.newPutProposal() + pd, data = pc.newPutProposal(hlc.Timestamp{}) } _, toks1[i] = b.TrackEvaluatingRequest(ctx, hlc.MinTimestamp) _, err := b.Insert(ctx, pd, data, toks1[i]) @@ -493,7 +496,7 @@ func TestProposalBufferRegistrationWithInsertionErrors(t *testing.T) { if i%2 == 0 { pd, data = pc.newLeaseProposal(roachpb.Lease{}) } else { - pd, data = pc.newPutProposal() + pd, data = pc.newPutProposal(hlc.Timestamp{}) } _, toks2[i] = b.TrackEvaluatingRequest(ctx, hlc.MinTimestamp) _, err := b.Insert(ctx, pd, data, toks2[i]) @@ -759,37 +762,48 @@ func TestProposalBufferClosedTimestamp(t *testing.T) { // proposed. lease roachpb.Lease + // expClosed is the expected closed timestamp carried by the proposal. Empty + // means the proposal is not expected to carry a closed timestamp update. expClosed hlc.Timestamp + // expAssignedClosedBumped, if set, means that the test expects + // b.assignedClosedTimestamp to be bumped before proposing. If not set, then + // the test expects b.assignedClosedTimestamp to be left at + // prevClosedTimestamp, regardless of whether the proposal carries a closed + // timestamp or not (expClosed). + expAssignedClosedBumped bool }{ { - name: "basic", - reqType: regularWrite, - trackerLowerBound: hlc.Timestamp{}, - leaseExp: hlc.MaxTimestamp, - rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, - prevClosedTimestamp: hlc.Timestamp{}, - expClosed: nowMinusClosedLag, + name: "basic", + reqType: regularWrite, + trackerLowerBound: hlc.Timestamp{}, + leaseExp: hlc.MaxTimestamp, + rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, + prevClosedTimestamp: hlc.Timestamp{}, + expClosed: nowMinusClosedLag, + expAssignedClosedBumped: true, }, { // The request tracker will prevent us from closing below its lower bound. - name: "not closing below evaluating requests", - reqType: regularWrite, - trackerLowerBound: nowMinusTwiceClosedLag, - leaseExp: hlc.MaxTimestamp, - rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, - prevClosedTimestamp: hlc.Timestamp{}, - expClosed: nowMinusTwiceClosedLag.FloorPrev(), + name: "not closing below evaluating requests", + reqType: regularWrite, + trackerLowerBound: nowMinusTwiceClosedLag, + leaseExp: hlc.MaxTimestamp, + rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, + prevClosedTimestamp: hlc.Timestamp{}, + expClosed: nowMinusTwiceClosedLag.FloorPrev(), + expAssignedClosedBumped: true, }, { // Like the basic test, except that we can't close timestamp below what // we've already closed previously. - name: "no regression", - reqType: regularWrite, - trackerLowerBound: hlc.Timestamp{}, - leaseExp: hlc.MaxTimestamp, - rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, - prevClosedTimestamp: someClosedTS, - expClosed: someClosedTS, + name: "no regression", + reqType: regularWrite, + trackerLowerBound: hlc.Timestamp{}, + leaseExp: hlc.MaxTimestamp, + rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, + prevClosedTimestamp: someClosedTS, + expClosed: someClosedTS, + expAssignedClosedBumped: false, }, { name: "brand new lease", @@ -805,6 +819,11 @@ func TestProposalBufferClosedTimestamp(t *testing.T) { leaseExp: expiredLeaseTimestamp, rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, expClosed: now.ToTimestamp(), + // Check that the lease proposal does not bump b.assignedClosedTimestamp. + // The proposer cannot make promises about the write timestamps of further + // requests based on the start time of a proposed lease. See comments in + // propBuf.assignClosedTimestampToProposalLocked(). + expAssignedClosedBumped: false, }, { name: "lease extension", @@ -821,7 +840,8 @@ func TestProposalBufferClosedTimestamp(t *testing.T) { rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, // Lease extensions don't carry closed timestamps because they don't get // MLAIs, and so they can be reordered. - expClosed: hlc.Timestamp{}, + expClosed: hlc.Timestamp{}, + expAssignedClosedBumped: false, }, { // Lease transfers behave just like regular writes. The lease start time @@ -832,21 +852,23 @@ func TestProposalBufferClosedTimestamp(t *testing.T) { Sequence: curLease.Sequence + 1, Start: now, }, - trackerLowerBound: hlc.Timestamp{}, - leaseExp: hlc.MaxTimestamp, - rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, - expClosed: nowMinusClosedLag, + trackerLowerBound: hlc.Timestamp{}, + leaseExp: hlc.MaxTimestamp, + rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, + expClosed: nowMinusClosedLag, + expAssignedClosedBumped: true, }, { // With the LEAD_FOR_GLOBAL_READS policy, we're expecting to close // timestamps in the future. - name: "global range", - reqType: regularWrite, - trackerLowerBound: hlc.Timestamp{}, - leaseExp: hlc.MaxTimestamp, - rangePolicy: roachpb.LEAD_FOR_GLOBAL_READS, - prevClosedTimestamp: hlc.Timestamp{}, - expClosed: nowPlusGlobalReadLead, + name: "global range", + reqType: regularWrite, + trackerLowerBound: hlc.Timestamp{}, + leaseExp: hlc.MaxTimestamp, + rangePolicy: roachpb.LEAD_FOR_GLOBAL_READS, + prevClosedTimestamp: hlc.Timestamp{}, + expClosed: nowPlusGlobalReadLead, + expAssignedClosedBumped: true, }, } { t.Run(tc.name, func(t *testing.T) { @@ -871,7 +893,7 @@ func TestProposalBufferClosedTimestamp(t *testing.T) { var data []byte switch tc.reqType { case regularWrite: - pd, data = pc.newPutProposal() + pd, data = pc.newPutProposal(now.ToTimestamp()) case newLease: pd, data = pc.newLeaseProposal(tc.lease) case leaseTransfer: @@ -896,6 +918,11 @@ func TestProposalBufferClosedTimestamp(t *testing.T) { require.NoError(t, err) require.NoError(t, b.flushLocked(ctx)) checkClosedTS(t, r, tc.expClosed) + if tc.expAssignedClosedBumped { + require.Equal(t, tc.expClosed, b.assignedClosedTimestamp) + } else { + require.Equal(t, tc.prevClosedTimestamp, b.assignedClosedTimestamp) + } }) } } diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index 59378df61af4..811aa512980e 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -47,6 +47,11 @@ type StoreTestingKnobs struct { // TestingProposalFilter is called before proposing each command. TestingProposalFilter kvserverbase.ReplicaProposalFilter + // TestingProposalSubmitFilter can be used by tests to observe and optionally + // drop Raft proposals before they are handed to etcd/raft to begin the + // process of replication. Dropped proposals are still eligible to be + // reproposed due to ticks. + TestingProposalSubmitFilter func(*ProposalData) (drop bool, err error) // TestingApplyFilter is called before applying the results of a // command on each replica. If it returns an error, the command will diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index e97cac76e34e..3149d6e89818 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -3298,7 +3298,9 @@ var xxx_messageInfo_TransferLeaseRequest proto.InternalMessageInfo // LeaseInfoRequest is the argument to the LeaseInfo() method, for getting // information about a range's lease. // It's a point request, so it addresses one single range, and returns the lease -// currently in effect for that range. +// currently in effect for that range. This request is commonly set with +// ReadConsistency=INCONSISTENT in order for the request to be served by the +// node to whom the request was sent. type LeaseInfoRequest struct { RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` } @@ -3338,6 +3340,19 @@ type LeaseInfoResponse struct { // The last lease known by the replica serving the request. It can also be the // tentative future lease, if a lease transfer is in progress. Lease Lease `protobuf:"bytes,2,opt,name=lease,proto3" json:"lease"` + // current_lease is set if `lease` represents a tentative future lease. In + // that case, current_lease represents the lease that's currently in effect. + CurrentLease *Lease `protobuf:"bytes,3,opt,name=current_lease,json=currentLease,proto3" json:"current_lease,omitempty"` + // evaluated_by returns the store that evaluated this request. This + // corresponds to the leaseholder unless ReadConsistency=INCONSISTENT was + // used. The response reflects the evaluator's view of the lease. When the + // client cares to see a particular node's view, it can use this field to + // check whether the node it intended query (by sending the request to that + // node and using ReadConsistency=INCONSISTENT) indeed served it - it's + // possible that even if ReadConsistency=INCONSISTENT was used, the request is + // still not evaluated by the node it was sent to if that node's replica is a + // learner or the node doesn't have a replica at all. + EvaluatedBy StoreID `protobuf:"varint,4,opt,name=evaluated_by,json=evaluatedBy,proto3,casttype=StoreID" json:"evaluated_by,omitempty"` } func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } @@ -7311,528 +7326,529 @@ func init() { func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_e08772acc330f58b) } var fileDescriptor_e08772acc330f58b = []byte{ - // 8322 bytes of a gzipped FileDescriptorProto + // 8352 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0xc9, - 0x76, 0x9e, 0x9a, 0xa4, 0x24, 0xf2, 0x50, 0xa2, 0x5a, 0xa5, 0xf9, 0xe1, 0x70, 0x77, 0x47, 0x33, - 0x3d, 0xff, 0x73, 0x77, 0xa9, 0x9d, 0x99, 0xbb, 0xd9, 0xf5, 0xce, 0x7a, 0xaf, 0x25, 0x8a, 0x33, - 0xa4, 0x34, 0xd2, 0x68, 0x9a, 0xd4, 0x0c, 0x76, 0x7d, 0x9d, 0x76, 0xab, 0xbb, 0x44, 0xf5, 0x15, - 0xd9, 0xcd, 0xe9, 0x6e, 0xea, 0x67, 0x81, 0x00, 0x71, 0x6c, 0x38, 0xf7, 0xc9, 0xb8, 0x01, 0x0c, - 0xf8, 0x1a, 0x0e, 0xe2, 0xeb, 0x5c, 0x23, 0x7e, 0x08, 0x90, 0x04, 0x48, 0x90, 0x3f, 0x24, 0xf6, - 0x4b, 0x80, 0x5c, 0x04, 0x4e, 0x7c, 0xfd, 0x66, 0x04, 0x88, 0x62, 0xeb, 0xe6, 0x21, 0x81, 0x83, - 0x20, 0xc8, 0x8b, 0x81, 0x7d, 0x08, 0x82, 0xfa, 0xe9, 0x3f, 0xb2, 0x49, 0x51, 0xb3, 0x7d, 0xe3, - 0x05, 0xfc, 0x22, 0xb1, 0x4f, 0xd5, 0x39, 0x5d, 0x75, 0xaa, 0xea, 0xd4, 0xf9, 0xaa, 0x4f, 0x55, - 0xc1, 0xbc, 0x6d, 0xa9, 0xda, 0x5e, 0x77, 0x67, 0x49, 0xed, 0x1a, 0xe5, 0xae, 0x6d, 0xb9, 0x16, - 0x9a, 0xd7, 0x2c, 0x6d, 0x9f, 0x92, 0xcb, 0x3c, 0xb1, 0x74, 0x7f, 0xff, 0x60, 0x69, 0xff, 0xc0, - 0xc1, 0xf6, 0x01, 0xb6, 0x97, 0x34, 0xcb, 0xd4, 0x7a, 0xb6, 0x8d, 0x4d, 0xed, 0x78, 0xa9, 0x6d, - 0x69, 0xfb, 0xf4, 0x8f, 0x61, 0xb6, 0x18, 0x7b, 0x34, 0xaf, 0x8d, 0x55, 0xdd, 0xe9, 0x75, 0x3a, - 0xaa, 0x7d, 0xbc, 0x64, 0x3b, 0xdd, 0x9d, 0x25, 0xfe, 0xc0, 0xf3, 0x22, 0xef, 0xed, 0xba, 0xea, - 0xaa, 0x9c, 0x76, 0xc1, 0xa3, 0x61, 0xdb, 0xb6, 0x6c, 0x87, 0x53, 0x2f, 0x79, 0xd4, 0x0e, 0x76, - 0xd5, 0x50, 0xee, 0xb7, 0x1c, 0xd7, 0xb2, 0xd5, 0x16, 0x5e, 0xc2, 0x66, 0xcb, 0x30, 0x31, 0xc9, - 0x70, 0xa0, 0x69, 0x3c, 0xf1, 0xed, 0xd8, 0xc4, 0x47, 0x3c, 0xb5, 0xd8, 0x73, 0x8d, 0xf6, 0xd2, - 0x5e, 0x5b, 0x5b, 0x72, 0x8d, 0x0e, 0x76, 0x5c, 0xb5, 0xd3, 0xf5, 0xaa, 0x40, 0x53, 0x5c, 0x5b, - 0xd5, 0x0c, 0xb3, 0xe5, 0xfd, 0xef, 0xee, 0x2c, 0xd9, 0x58, 0xb3, 0x6c, 0x1d, 0xeb, 0x8a, 0xd3, - 0x55, 0x4d, 0xaf, 0xb8, 0x2d, 0xab, 0x65, 0xd1, 0x9f, 0x4b, 0xe4, 0x17, 0xa7, 0x5e, 0x6d, 0x59, - 0x56, 0xab, 0x8d, 0x97, 0xe8, 0xd3, 0x4e, 0x6f, 0x77, 0x49, 0xef, 0xd9, 0xaa, 0x6b, 0x58, 0x9c, - 0x4b, 0xfa, 0xe7, 0x02, 0xcc, 0xca, 0xf8, 0x75, 0x0f, 0x3b, 0x6e, 0x0d, 0xab, 0x3a, 0xb6, 0xd1, - 0x15, 0x48, 0xef, 0xe3, 0xe3, 0x62, 0xfa, 0x9a, 0x70, 0x77, 0x66, 0x65, 0xfa, 0xcb, 0x93, 0xc5, - 0xf4, 0x3a, 0x3e, 0x96, 0x09, 0x0d, 0x5d, 0x83, 0x69, 0x6c, 0xea, 0x0a, 0x49, 0xce, 0x44, 0x93, - 0xa7, 0xb0, 0xa9, 0xaf, 0xe3, 0x63, 0xf4, 0x6d, 0xc8, 0x3a, 0x44, 0x9a, 0xa9, 0xe1, 0xe2, 0xe4, - 0x35, 0xe1, 0xee, 0xe4, 0xca, 0xcf, 0x7d, 0x79, 0xb2, 0xf8, 0x49, 0xcb, 0x70, 0xf7, 0x7a, 0x3b, - 0x65, 0xcd, 0xea, 0x2c, 0xf9, 0x6d, 0xaa, 0xef, 0x04, 0xbf, 0x97, 0xba, 0xfb, 0xad, 0xa5, 0x7e, - 0x1d, 0x95, 0x9b, 0x47, 0x66, 0x03, 0xbf, 0x96, 0x7d, 0x89, 0x6b, 0x99, 0xac, 0x20, 0xa6, 0xd6, - 0x32, 0xd9, 0x94, 0x98, 0x96, 0xfe, 0x28, 0x05, 0x05, 0x19, 0x3b, 0x5d, 0xcb, 0x74, 0x30, 0x2f, - 0xf9, 0xfb, 0x90, 0x76, 0x8f, 0x4c, 0x5a, 0xf2, 0xfc, 0xc3, 0xab, 0xe5, 0x81, 0xde, 0x53, 0x6e, - 0xda, 0xaa, 0xe9, 0xa8, 0x1a, 0xa9, 0xbe, 0x4c, 0xb2, 0xa2, 0x8f, 0x20, 0x6f, 0x63, 0xa7, 0xd7, - 0xc1, 0x54, 0x91, 0xb4, 0x52, 0xf9, 0x87, 0x97, 0x63, 0x38, 0x1b, 0x5d, 0xd5, 0x94, 0x81, 0xe5, - 0x25, 0xbf, 0x51, 0x03, 0x66, 0x39, 0xa7, 0x8d, 0x55, 0xc7, 0x32, 0x8b, 0xd3, 0xd7, 0x84, 0xbb, - 0x85, 0x87, 0xe5, 0x18, 0xde, 0x68, 0x29, 0xc9, 0x63, 0xaf, 0x83, 0x65, 0xca, 0x25, 0xcf, 0xd8, - 0xa1, 0x27, 0x74, 0x05, 0xb2, 0x66, 0xaf, 0x43, 0xf4, 0xeb, 0x50, 0xed, 0xa5, 0xe5, 0x69, 0xb3, - 0xd7, 0x59, 0xc7, 0xc7, 0x0e, 0x7a, 0x0b, 0x72, 0x24, 0x69, 0xe7, 0xd8, 0xc5, 0x4e, 0x31, 0x4b, - 0xd3, 0x48, 0xde, 0x15, 0xf2, 0x2c, 0x7d, 0x0a, 0x33, 0x61, 0xa9, 0x08, 0x41, 0x41, 0xae, 0x36, - 0xb6, 0x37, 0xaa, 0xca, 0xf6, 0xe6, 0xfa, 0xe6, 0xf3, 0x57, 0x9b, 0xe2, 0x04, 0xba, 0x00, 0x22, - 0xa7, 0xad, 0x57, 0x3f, 0x53, 0x9e, 0xd5, 0x37, 0xea, 0x4d, 0x51, 0x28, 0x65, 0xbe, 0xfb, 0xc3, - 0xab, 0x13, 0x6b, 0x99, 0xec, 0x94, 0x38, 0x2d, 0xfd, 0x50, 0x00, 0x78, 0x8a, 0x5d, 0xde, 0x1b, - 0xd0, 0x0a, 0x4c, 0xed, 0xd1, 0x12, 0x17, 0x05, 0xaa, 0x96, 0x6b, 0xb1, 0x55, 0x0b, 0xf5, 0x9c, - 0x95, 0xec, 0x8f, 0x4e, 0x16, 0x27, 0x7e, 0x7c, 0xb2, 0x28, 0xc8, 0x9c, 0x13, 0xbd, 0x80, 0xfc, - 0x3e, 0x3e, 0x56, 0xf8, 0xb8, 0x2c, 0xa6, 0xa8, 0x8e, 0xde, 0x0f, 0x09, 0xda, 0x3f, 0x28, 0x7b, - 0x43, 0xb4, 0x1c, 0x1a, 0xce, 0x65, 0xc2, 0x51, 0x6e, 0xb8, 0x36, 0x36, 0x5b, 0xee, 0x9e, 0x0c, - 0xfb, 0xf8, 0xf8, 0x19, 0x93, 0x21, 0xfd, 0x81, 0x00, 0x79, 0x5a, 0x4a, 0xa6, 0x54, 0x54, 0xe9, - 0x2b, 0xe6, 0xf5, 0x33, 0x5b, 0x20, 0xa6, 0x9c, 0x65, 0x98, 0x3c, 0x50, 0xdb, 0x3d, 0x4c, 0x4b, - 0x98, 0x7f, 0x58, 0x8c, 0x91, 0xf1, 0x92, 0xa4, 0xcb, 0x2c, 0x1b, 0x7a, 0x0c, 0x33, 0x86, 0xe9, - 0x62, 0xd3, 0x55, 0x18, 0x5b, 0xfa, 0x0c, 0xb6, 0x3c, 0xcb, 0x4d, 0x1f, 0xa4, 0x7f, 0x26, 0x00, - 0x6c, 0xf5, 0x12, 0xd5, 0xf3, 0x37, 0xc7, 0x2c, 0xff, 0x4a, 0x86, 0xb0, 0x7a, 0xb5, 0xb8, 0x04, - 0x53, 0x86, 0xd9, 0x36, 0x4c, 0x56, 0xfe, 0xac, 0xcc, 0x9f, 0xd0, 0x05, 0x98, 0xdc, 0x69, 0x1b, - 0xa6, 0x4e, 0xc7, 0x43, 0x56, 0x66, 0x0f, 0x92, 0x0c, 0x79, 0x5a, 0xea, 0x04, 0xf5, 0x2e, 0x9d, - 0xa4, 0xe0, 0x62, 0xc5, 0x32, 0x75, 0x83, 0x0c, 0x49, 0xb5, 0xfd, 0xb5, 0xd0, 0xca, 0x1a, 0x5c, - 0xd0, 0x71, 0xd7, 0xc6, 0x9a, 0xea, 0x62, 0x5d, 0xc1, 0x47, 0xdd, 0x31, 0xdb, 0x18, 0x05, 0x5c, - 0xd5, 0xa3, 0x2e, 0xa5, 0x91, 0x51, 0x4b, 0x04, 0xb0, 0x51, 0x3b, 0x45, 0x4c, 0xa6, 0x9c, 0xc5, - 0x47, 0x5d, 0x3a, 0x6a, 0xe3, 0xd5, 0x8c, 0xbe, 0x09, 0x97, 0xd5, 0x76, 0xdb, 0x3a, 0x54, 0x8c, - 0x5d, 0x45, 0xb7, 0xb0, 0xa3, 0x98, 0x96, 0xab, 0xe0, 0x23, 0xc3, 0x71, 0xa9, 0x49, 0xc8, 0xca, - 0x0b, 0x34, 0xb9, 0xbe, 0xbb, 0x6a, 0x61, 0x67, 0xd3, 0x72, 0xab, 0x24, 0x29, 0xd4, 0x94, 0xd3, - 0xe1, 0xa6, 0x94, 0x7e, 0x01, 0x2e, 0xf5, 0xeb, 0x37, 0xc9, 0xf6, 0xfb, 0x43, 0x01, 0x0a, 0x75, - 0xd3, 0x70, 0xbf, 0x16, 0x0d, 0xe7, 0xeb, 0x33, 0x1d, 0xd6, 0xe7, 0x7d, 0x10, 0x77, 0x55, 0xa3, - 0xfd, 0xdc, 0x6c, 0x5a, 0x9d, 0x1d, 0xc7, 0xb5, 0x4c, 0xec, 0x70, 0x85, 0x0f, 0xd0, 0xa5, 0x97, - 0x30, 0xe7, 0xd7, 0x26, 0x49, 0x35, 0xb9, 0x20, 0xd6, 0x4d, 0xcd, 0xc6, 0x1d, 0x6c, 0x26, 0xaa, - 0xa7, 0xb7, 0x21, 0x67, 0x78, 0x72, 0xa9, 0xae, 0xd2, 0x72, 0x40, 0x90, 0x7a, 0x30, 0x1f, 0x7a, - 0x6b, 0x92, 0xe6, 0x92, 0x4c, 0x46, 0xf8, 0x50, 0x09, 0xda, 0x88, 0x4c, 0x46, 0xf8, 0x90, 0x99, - 0xb7, 0x06, 0xcc, 0xae, 0xe2, 0x36, 0x76, 0x71, 0x82, 0x35, 0x95, 0xb6, 0xa1, 0xe0, 0x09, 0x4d, - 0xb2, 0x61, 0x7e, 0x43, 0x00, 0xc4, 0xe5, 0xaa, 0x66, 0x2b, 0xc9, 0x12, 0xa3, 0x45, 0xe2, 0x5a, - 0xb8, 0x3d, 0xdb, 0x64, 0xd3, 0x39, 0xeb, 0x93, 0xc0, 0x48, 0x74, 0x46, 0x0f, 0x86, 0x6c, 0x26, - 0x3c, 0x64, 0xb9, 0x7b, 0x73, 0x08, 0x0b, 0x91, 0x82, 0x25, 0xdb, 0x7c, 0x19, 0x5a, 0xa6, 0xd4, - 0xb5, 0x74, 0xd8, 0x87, 0xa3, 0x44, 0xe9, 0xfb, 0x02, 0xcc, 0x57, 0xda, 0x58, 0xb5, 0x13, 0xd7, - 0xc8, 0xb7, 0x20, 0xab, 0x63, 0x55, 0xa7, 0x55, 0x66, 0x03, 0xfb, 0x9d, 0x90, 0x14, 0xe2, 0xe9, - 0x96, 0xf7, 0xda, 0x5a, 0xb9, 0xe9, 0xf9, 0xc0, 0x7c, 0x74, 0xfb, 0x4c, 0xd2, 0x67, 0x80, 0xc2, - 0x25, 0x4b, 0xb2, 0x23, 0xfc, 0x6e, 0x0a, 0x90, 0x8c, 0x0f, 0xb0, 0xed, 0x26, 0x5e, 0xed, 0x55, - 0xc8, 0xbb, 0xaa, 0xdd, 0xc2, 0xae, 0x42, 0xbc, 0xfb, 0xf3, 0xd4, 0x1c, 0x18, 0x1f, 0x21, 0xa3, - 0x26, 0xdc, 0xc1, 0xa6, 0xba, 0xd3, 0xc6, 0x54, 0x8a, 0xb2, 0x63, 0xf5, 0x4c, 0x5d, 0x31, 0x5c, - 0x6c, 0xab, 0xae, 0x65, 0x2b, 0x56, 0xd7, 0x35, 0x3a, 0xc6, 0x17, 0xd4, 0xb1, 0xe7, 0x5d, 0xed, - 0x06, 0xcb, 0x4e, 0x98, 0x57, 0x48, 0xe6, 0x3a, 0xcf, 0xfb, 0x3c, 0x94, 0x15, 0x95, 0x61, 0xc1, - 0x68, 0x99, 0x96, 0x8d, 0x95, 0x96, 0xa6, 0xb8, 0x7b, 0x36, 0x76, 0xf6, 0xac, 0xb6, 0x37, 0x21, - 0xcd, 0xb3, 0xa4, 0xa7, 0x5a, 0xd3, 0x4b, 0x90, 0x3e, 0x87, 0x85, 0x88, 0x96, 0x92, 0x6c, 0x82, - 0xff, 0x2d, 0x40, 0xbe, 0xa1, 0xa9, 0x66, 0x92, 0xba, 0xff, 0x14, 0xf2, 0x8e, 0xa6, 0x9a, 0xca, - 0xae, 0x65, 0x77, 0x54, 0x97, 0xd6, 0xab, 0x10, 0xd1, 0xbd, 0xef, 0xdf, 0x6b, 0xaa, 0xf9, 0x84, - 0x66, 0x92, 0xc1, 0xf1, 0x7f, 0xf7, 0xfb, 0xaf, 0x93, 0x5f, 0xdd, 0x7f, 0x65, 0xc3, 0x7b, 0x2d, - 0x93, 0x4d, 0x8b, 0x19, 0xe9, 0x2f, 0x04, 0x98, 0x61, 0x55, 0x4e, 0x72, 0x78, 0x7f, 0x00, 0x19, - 0xdb, 0x3a, 0x64, 0xc3, 0x3b, 0xff, 0xf0, 0xad, 0x18, 0x11, 0xeb, 0xf8, 0x38, 0x3c, 0x7f, 0xd2, - 0xec, 0x68, 0x05, 0xb8, 0x97, 0xaa, 0x50, 0xee, 0xf4, 0xb8, 0xdc, 0xc0, 0xb8, 0x64, 0x22, 0xe3, - 0x0e, 0xcc, 0xed, 0xa8, 0xae, 0xb6, 0xa7, 0xd8, 0xbc, 0x90, 0x64, 0xae, 0x4d, 0xdf, 0x9d, 0x91, - 0x0b, 0x94, 0xec, 0x15, 0xdd, 0x21, 0x35, 0x67, 0xe3, 0xcd, 0xc1, 0x7f, 0xc5, 0xda, 0xfc, 0xff, - 0x0a, 0x7c, 0x0c, 0x79, 0x35, 0xff, 0xab, 0xd6, 0xf4, 0xbf, 0x99, 0x82, 0xcb, 0x95, 0x3d, 0xac, - 0xed, 0x57, 0x2c, 0xd3, 0x31, 0x1c, 0x97, 0xe8, 0x2e, 0xc9, 0xf6, 0x7f, 0x0b, 0x72, 0x87, 0x86, - 0xbb, 0xa7, 0xe8, 0xc6, 0xee, 0x2e, 0xb5, 0xb6, 0x59, 0x39, 0x4b, 0x08, 0xab, 0xc6, 0xee, 0x2e, - 0x7a, 0x04, 0x99, 0x8e, 0xa5, 0x33, 0x67, 0xbe, 0xf0, 0x70, 0x31, 0x46, 0x3c, 0x2d, 0x9a, 0xd3, - 0xeb, 0x6c, 0x58, 0x3a, 0x96, 0x69, 0x66, 0x74, 0x15, 0x40, 0x23, 0xd4, 0xae, 0x65, 0x98, 0x2e, - 0x37, 0x8e, 0x21, 0x0a, 0xaa, 0x41, 0xce, 0xc5, 0x76, 0xc7, 0x30, 0x55, 0x17, 0x17, 0x27, 0xa9, - 0xf2, 0x6e, 0xc6, 0x16, 0xbc, 0xdb, 0x36, 0x34, 0x75, 0x15, 0x3b, 0x9a, 0x6d, 0x74, 0x5d, 0xcb, - 0xe6, 0x5a, 0x0c, 0x98, 0xa5, 0x5f, 0xcb, 0x40, 0x71, 0x50, 0x37, 0x49, 0xf6, 0x90, 0x2d, 0x98, - 0xb2, 0xb1, 0xd3, 0x6b, 0xbb, 0xbc, 0x8f, 0x3c, 0x1c, 0xa6, 0x82, 0x98, 0x12, 0xd0, 0xa5, 0x8b, - 0xb6, 0xcb, 0x8b, 0xcd, 0xe5, 0x94, 0xfe, 0x8d, 0x00, 0x53, 0x2c, 0x01, 0x3d, 0x80, 0xac, 0x4d, - 0x26, 0x06, 0xc5, 0xd0, 0x69, 0x19, 0xd3, 0x2b, 0x97, 0x4e, 0x4f, 0x16, 0xa7, 0xe9, 0x64, 0x51, - 0x5f, 0xfd, 0x32, 0xf8, 0x29, 0x4f, 0xd3, 0x7c, 0x75, 0x9d, 0xb4, 0x96, 0xe3, 0xaa, 0xb6, 0x4b, - 0x17, 0x95, 0x52, 0x0c, 0x21, 0x51, 0xc2, 0x3a, 0x3e, 0x46, 0x6b, 0x30, 0xe5, 0xb8, 0xaa, 0xdb, - 0x73, 0x78, 0x7b, 0x9d, 0xab, 0xb0, 0x0d, 0xca, 0x29, 0x73, 0x09, 0xc4, 0xdd, 0xd2, 0xb1, 0xab, - 0x1a, 0x6d, 0xda, 0x80, 0x39, 0x99, 0x3f, 0x49, 0xbf, 0x25, 0xc0, 0x14, 0xcb, 0x8a, 0x2e, 0xc3, - 0x82, 0xbc, 0xbc, 0xf9, 0xb4, 0xaa, 0xd4, 0x37, 0x57, 0xab, 0xcd, 0xaa, 0xbc, 0x51, 0xdf, 0x5c, - 0x6e, 0x56, 0xc5, 0x09, 0x74, 0x09, 0x90, 0x97, 0x50, 0x79, 0xbe, 0xd9, 0xa8, 0x37, 0x9a, 0xd5, - 0xcd, 0xa6, 0x28, 0xd0, 0x35, 0x15, 0x4a, 0x0f, 0x51, 0x53, 0xe8, 0x26, 0x5c, 0xeb, 0xa7, 0x2a, - 0x8d, 0xe6, 0x72, 0xb3, 0xa1, 0x54, 0x1b, 0xcd, 0xfa, 0xc6, 0x72, 0xb3, 0xba, 0x2a, 0xa6, 0x47, - 0xe4, 0x22, 0x2f, 0x91, 0xe5, 0x6a, 0xa5, 0x29, 0x66, 0x24, 0x17, 0x2e, 0xca, 0x58, 0xb3, 0x3a, - 0xdd, 0x9e, 0x8b, 0x49, 0x29, 0x9d, 0x24, 0x47, 0xca, 0x65, 0x98, 0xd6, 0xed, 0x63, 0xc5, 0xee, - 0x99, 0x7c, 0x9c, 0x4c, 0xe9, 0xf6, 0xb1, 0xdc, 0x33, 0xa5, 0x7f, 0x22, 0xc0, 0xa5, 0xfe, 0xd7, - 0x26, 0xd9, 0x09, 0x5f, 0x40, 0x5e, 0xd5, 0x75, 0xac, 0x2b, 0x3a, 0x6e, 0xbb, 0x2a, 0x77, 0x89, - 0xee, 0x87, 0x24, 0xf1, 0xa5, 0xc0, 0xb2, 0xbf, 0x14, 0xb8, 0xf1, 0xb2, 0x52, 0xa1, 0x05, 0x59, - 0x25, 0x1c, 0x9e, 0xf9, 0xa1, 0x42, 0x28, 0x45, 0xfa, 0x9f, 0x19, 0x98, 0xad, 0x9a, 0x7a, 0xf3, - 0x28, 0xd1, 0xb9, 0xe4, 0x12, 0x4c, 0x69, 0x56, 0xa7, 0x63, 0xb8, 0x9e, 0x82, 0xd8, 0x13, 0xfa, - 0x99, 0x90, 0x2b, 0x9b, 0x1e, 0xc3, 0xa1, 0x0b, 0x9c, 0x58, 0xf4, 0x8b, 0x70, 0x99, 0x58, 0x4d, - 0xdb, 0x54, 0xdb, 0x0a, 0x93, 0xa6, 0xb8, 0xb6, 0xd1, 0x6a, 0x61, 0x9b, 0x2f, 0x3f, 0xde, 0x8d, - 0x29, 0x67, 0x9d, 0x73, 0x54, 0x28, 0x43, 0x93, 0xe5, 0x97, 0x2f, 0x1a, 0x71, 0x64, 0xf4, 0x09, - 0x00, 0x99, 0x8a, 0xe8, 0x92, 0xa6, 0xc3, 0xed, 0xd1, 0xb0, 0x35, 0x4d, 0xcf, 0x04, 0x11, 0x06, - 0xf2, 0xec, 0xa0, 0x17, 0x20, 0x1a, 0xa6, 0xb2, 0xdb, 0x36, 0x5a, 0x7b, 0xae, 0x72, 0x68, 0x1b, - 0x2e, 0x76, 0x8a, 0xf3, 0x54, 0x46, 0x5c, 0x53, 0x37, 0xf8, 0xd2, 0xac, 0xfe, 0x8a, 0xe4, 0xe4, - 0xd2, 0x0a, 0x86, 0xf9, 0x84, 0xf2, 0x53, 0xa2, 0x83, 0x96, 0x08, 0x14, 0x7a, 0xdd, 0x33, 0x6c, - 0xac, 0x3c, 0xe8, 0x6a, 0x74, 0x1d, 0x24, 0xbb, 0x52, 0x38, 0x3d, 0x59, 0x04, 0x99, 0x91, 0x1f, - 0x6c, 0x55, 0x08, 0x34, 0x62, 0xbf, 0xbb, 0x1a, 0x7a, 0x05, 0xf7, 0x42, 0x4b, 0x30, 0x64, 0x32, - 0xe7, 0x9a, 0x52, 0x5d, 0x65, 0xcf, 0x68, 0xed, 0x61, 0x5b, 0xf1, 0x57, 0xca, 0xe9, 0x62, 0x68, - 0x56, 0xbe, 0x19, 0x30, 0x54, 0x54, 0x93, 0x29, 0x64, 0xd9, 0xad, 0xd1, 0xcc, 0x7e, 0x33, 0x90, - 0xf6, 0xec, 0x5a, 0x86, 0x63, 0x99, 0xc5, 0x1c, 0x6b, 0x4f, 0xf6, 0x84, 0xee, 0x81, 0xe8, 0x1e, - 0x99, 0xca, 0x1e, 0x56, 0x6d, 0x77, 0x07, 0xab, 0x2e, 0x99, 0xf8, 0x81, 0xe6, 0x98, 0x73, 0x8f, - 0xcc, 0x5a, 0x88, 0xbc, 0x96, 0xc9, 0x4e, 0x8b, 0x59, 0xe9, 0xbf, 0x08, 0x50, 0xf0, 0xba, 0x5b, - 0x92, 0x23, 0xe3, 0x2e, 0x88, 0x96, 0x89, 0x95, 0xee, 0x9e, 0xea, 0x60, 0x5e, 0x69, 0x3e, 0xe1, - 0x14, 0x2c, 0x13, 0x6f, 0x11, 0x32, 0xab, 0x1b, 0xda, 0x82, 0x79, 0xc7, 0x55, 0x5b, 0x86, 0xd9, - 0x0a, 0xe9, 0x62, 0x72, 0x7c, 0x70, 0x21, 0x72, 0x6e, 0x9f, 0x1e, 0xf1, 0x52, 0xfe, 0x58, 0x80, - 0xf9, 0x65, 0xbd, 0x63, 0x98, 0x8d, 0x6e, 0xdb, 0x48, 0x74, 0xcd, 0xe2, 0x26, 0xe4, 0x1c, 0x22, - 0x33, 0x30, 0xf8, 0x01, 0x02, 0xcd, 0xd2, 0x14, 0x62, 0xf9, 0x9f, 0xc1, 0x1c, 0x3e, 0xea, 0x1a, - 0xec, 0x53, 0x05, 0x03, 0x4e, 0x99, 0xf1, 0xeb, 0x56, 0x08, 0x78, 0x49, 0x12, 0xaf, 0xd3, 0x67, - 0x80, 0xc2, 0x55, 0x4a, 0x12, 0xbb, 0x7c, 0x06, 0x0b, 0x54, 0xf4, 0xb6, 0xe9, 0x24, 0xac, 0x2f, - 0xe9, 0xe7, 0xe1, 0x42, 0x54, 0x74, 0x92, 0xe5, 0x7e, 0xc5, 0x5b, 0x79, 0x03, 0xdb, 0x89, 0x82, - 0x5e, 0x5f, 0xd7, 0x5c, 0x70, 0x92, 0x65, 0xfe, 0x15, 0x01, 0xae, 0x50, 0xd9, 0xf4, 0x6b, 0xce, - 0x2e, 0xb6, 0x9f, 0x61, 0xd5, 0x49, 0x14, 0xb1, 0xdf, 0x80, 0x29, 0x86, 0xbc, 0x69, 0xff, 0x9c, - 0x5c, 0xc9, 0x13, 0xcf, 0xa5, 0xe1, 0x5a, 0x36, 0xf1, 0x5c, 0x78, 0x92, 0xa4, 0x42, 0x29, 0xae, - 0x14, 0x49, 0xd6, 0xf4, 0xef, 0x09, 0x30, 0xcf, 0x9d, 0x46, 0xd2, 0x95, 0x2b, 0x7b, 0xc4, 0x67, - 0x42, 0x55, 0xc8, 0x6b, 0xf4, 0x97, 0xe2, 0x1e, 0x77, 0x31, 0x95, 0x5f, 0x18, 0xe5, 0x6f, 0x32, - 0xb6, 0xe6, 0x71, 0x17, 0x13, 0xa7, 0xd5, 0xfb, 0x4d, 0x14, 0x15, 0xaa, 0xe4, 0x48, 0x8f, 0x95, - 0x8e, 0x23, 0x9a, 0xd7, 0x73, 0xfd, 0xb8, 0x0e, 0xfe, 0x69, 0x9a, 0x2b, 0x81, 0xbd, 0x83, 0x67, - 0x4f, 0xd4, 0x47, 0xf9, 0x1c, 0x2e, 0x85, 0xa7, 0x82, 0x50, 0xc5, 0x53, 0xe7, 0xa8, 0x78, 0x68, - 0x45, 0x3f, 0xa0, 0xa2, 0xcf, 0x20, 0xb4, 0x66, 0xaf, 0xb0, 0x3a, 0x79, 0xe8, 0xe7, 0x3c, 0xea, - 0x98, 0x0f, 0xa4, 0x30, 0xba, 0x83, 0x2a, 0x90, 0xc5, 0x47, 0x5d, 0x45, 0xc7, 0x8e, 0xc6, 0x0d, - 0x97, 0x14, 0x27, 0x90, 0x14, 0x65, 0x00, 0x0f, 0x4c, 0xe3, 0xa3, 0x2e, 0x21, 0xa2, 0x6d, 0x32, - 0x15, 0x7b, 0xae, 0x02, 0x2d, 0xb6, 0x73, 0x36, 0xbc, 0x08, 0x7a, 0x0a, 0x17, 0x37, 0xe7, 0x7b, - 0x09, 0x4c, 0x84, 0xf4, 0x03, 0x01, 0xde, 0x8a, 0x6d, 0xb5, 0x24, 0x27, 0xb2, 0x4f, 0x20, 0x43, - 0x2b, 0x9f, 0x3a, 0x67, 0xe5, 0x29, 0x97, 0xf4, 0xdd, 0x14, 0x1f, 0xe3, 0x32, 0x6e, 0x5b, 0x44, - 0xb1, 0x89, 0xaf, 0xca, 0x3d, 0x87, 0xd9, 0x03, 0xcb, 0x25, 0x8e, 0x04, 0x6f, 0xf6, 0xd4, 0xb9, - 0x9b, 0x7d, 0x86, 0x0a, 0xf0, 0x5a, 0xfc, 0x25, 0xcc, 0x9b, 0x96, 0xa9, 0x44, 0x85, 0x9e, 0xbf, - 0x2f, 0xcd, 0x99, 0x96, 0xf9, 0x32, 0x24, 0xd7, 0xb7, 0x33, 0x7d, 0x9a, 0x48, 0xd2, 0xce, 0x7c, - 0x4f, 0x80, 0x05, 0xdf, 0xc7, 0x49, 0xd8, 0x83, 0xfe, 0x00, 0xd2, 0xa6, 0x75, 0x78, 0x9e, 0x55, - 0x4f, 0x92, 0x9f, 0xcc, 0x7a, 0xd1, 0x12, 0x25, 0x59, 0xdf, 0x7f, 0x9b, 0x82, 0xdc, 0xd3, 0x4a, - 0x92, 0xb5, 0xfc, 0x84, 0xaf, 0xa8, 0xb3, 0xf6, 0x8e, 0xeb, 0xed, 0xfe, 0xfb, 0xca, 0x4f, 0x2b, - 0xeb, 0xf8, 0xd8, 0xeb, 0xed, 0x84, 0x0b, 0x2d, 0x43, 0x2e, 0xba, 0xf6, 0x3a, 0xa6, 0xa6, 0x02, - 0xae, 0x12, 0x86, 0x49, 0x2a, 0xd7, 0x8b, 0xde, 0x10, 0x62, 0xa2, 0x37, 0xc8, 0x6b, 0x7c, 0x4f, - 0x31, 0x75, 0x9e, 0xd7, 0x84, 0x5c, 0xc4, 0x49, 0x71, 0x4a, 0x7a, 0x01, 0x40, 0xaa, 0x93, 0x64, - 0x93, 0xfc, 0x6a, 0x1a, 0x0a, 0x5b, 0x3d, 0x67, 0x2f, 0xe1, 0xde, 0x57, 0x01, 0xe8, 0xf6, 0x1c, - 0x8a, 0x17, 0x8e, 0x4c, 0x5e, 0xe7, 0x33, 0x02, 0x43, 0xbc, 0x4a, 0x33, 0xbe, 0xe6, 0x91, 0x89, - 0x6a, 0x5c, 0x08, 0x56, 0x82, 0xe8, 0x92, 0x1b, 0xa3, 0xc0, 0x6a, 0xf3, 0xc8, 0xdc, 0xc0, 0x3e, - 0x4a, 0x65, 0x92, 0x30, 0x91, 0xf4, 0x09, 0x4c, 0x93, 0x07, 0xc5, 0xb5, 0xce, 0xd3, 0xcc, 0x53, - 0x84, 0xa7, 0x69, 0xa1, 0xc7, 0x90, 0x63, 0xdc, 0x64, 0xf6, 0x9b, 0xa2, 0xb3, 0x5f, 0x5c, 0x5d, - 0xb8, 0x1a, 0xe9, 0xbc, 0x97, 0xa5, 0xac, 0x64, 0xae, 0xbb, 0x00, 0x93, 0xbb, 0x96, 0xad, 0x79, - 0xdf, 0x87, 0xd9, 0x03, 0x6b, 0xcf, 0xb5, 0x4c, 0x36, 0x2b, 0xe6, 0xd6, 0x32, 0xd9, 0x9c, 0x08, - 0xd2, 0x6f, 0x09, 0x30, 0xe7, 0x37, 0x44, 0x92, 0x13, 0x42, 0x25, 0xa2, 0xc5, 0xf3, 0x37, 0x05, - 0x51, 0xa0, 0xf4, 0xef, 0xa9, 0x47, 0xa4, 0x59, 0x07, 0xb4, 0x65, 0x92, 0xec, 0x29, 0x8f, 0x59, - 0xec, 0x50, 0xea, 0xbc, 0xad, 0x4b, 0xc3, 0x88, 0x1e, 0xc0, 0x05, 0xa3, 0x43, 0xec, 0xb9, 0xe1, - 0xb6, 0x8f, 0x39, 0x6c, 0x73, 0xb1, 0xf7, 0x21, 0x7a, 0x21, 0x48, 0xab, 0x78, 0x49, 0xd2, 0xef, - 0xd2, 0x05, 0xf0, 0xa0, 0x26, 0x49, 0xaa, 0xba, 0x0e, 0xb3, 0x36, 0x13, 0x4d, 0xdc, 0x9a, 0x73, - 0x6a, 0x7b, 0xc6, 0x67, 0x25, 0x0a, 0xff, 0x9d, 0x14, 0xcc, 0xbd, 0xe8, 0x61, 0xfb, 0xf8, 0xeb, - 0xa4, 0xee, 0xdb, 0x30, 0x77, 0xa8, 0x1a, 0xae, 0xb2, 0x6b, 0xd9, 0x4a, 0xaf, 0xab, 0xab, 0xae, - 0x17, 0xc0, 0x32, 0x4b, 0xc8, 0x4f, 0x2c, 0x7b, 0x9b, 0x12, 0x11, 0x06, 0xb4, 0x6f, 0x5a, 0x87, - 0xa6, 0x42, 0xc8, 0x14, 0x28, 0x1f, 0x99, 0x7c, 0x55, 0x7a, 0xe5, 0xc3, 0xff, 0x7c, 0xb2, 0xf8, - 0x68, 0xac, 0xb0, 0x34, 0x1a, 0x82, 0xd7, 0xeb, 0x19, 0x7a, 0x79, 0x7b, 0xbb, 0xbe, 0x2a, 0x8b, - 0x54, 0xe4, 0x2b, 0x26, 0xb1, 0x79, 0x64, 0x3a, 0xd2, 0x3f, 0x48, 0x81, 0x18, 0xe8, 0x28, 0xc9, - 0x86, 0xac, 0x42, 0xfe, 0x75, 0x0f, 0xdb, 0xc6, 0x1b, 0x34, 0x23, 0x70, 0x46, 0x62, 0x76, 0xee, - 0xc3, 0xbc, 0x7b, 0x64, 0x2a, 0x2c, 0x68, 0x90, 0xc5, 0x92, 0x78, 0x31, 0x10, 0x73, 0x2e, 0x29, - 0x33, 0xa1, 0xd3, 0x38, 0x12, 0x07, 0x7d, 0x0e, 0x33, 0x11, 0x6d, 0xa5, 0xbf, 0x9a, 0xb6, 0xf2, - 0x87, 0x21, 0x45, 0xfd, 0x81, 0x00, 0x88, 0x2a, 0xaa, 0xce, 0x3e, 0x1b, 0x7c, 0x5d, 0xfa, 0xd3, - 0x5d, 0x10, 0x69, 0x88, 0xa7, 0x62, 0xec, 0x2a, 0x1d, 0xc3, 0x71, 0x0c, 0xb3, 0xc5, 0x3b, 0x54, - 0x81, 0xd2, 0xeb, 0xbb, 0x1b, 0x8c, 0x2a, 0xfd, 0x0d, 0x58, 0x88, 0x54, 0x20, 0xc9, 0xc6, 0xbe, - 0x0e, 0x33, 0xbb, 0xec, 0xab, 0x2e, 0x15, 0xce, 0x57, 0x1c, 0xf3, 0x94, 0xc6, 0xde, 0x27, 0xfd, - 0x79, 0x0a, 0x2e, 0xc8, 0xd8, 0xb1, 0xda, 0x07, 0x38, 0x79, 0x15, 0xd6, 0x80, 0x7f, 0xce, 0x51, - 0xde, 0x48, 0x93, 0x39, 0xc6, 0xcc, 0xa6, 0xb9, 0xe8, 0xb2, 0xfd, 0xcd, 0xd1, 0x3d, 0x76, 0x70, - 0xa1, 0x9e, 0xaf, 0xd1, 0x65, 0x22, 0x6b, 0x74, 0x16, 0xcc, 0xb1, 0x0f, 0xd2, 0xba, 0xe2, 0xe0, - 0xd7, 0x66, 0xaf, 0xe3, 0x81, 0xa1, 0xf2, 0xa8, 0x42, 0xd6, 0x19, 0x4b, 0x03, 0xbf, 0xde, 0xec, - 0x75, 0xa8, 0xef, 0xbc, 0x72, 0x89, 0x94, 0xf7, 0xf4, 0x64, 0xb1, 0x10, 0x49, 0x73, 0xe4, 0x82, - 0xe1, 0x3f, 0x13, 0xe9, 0xd2, 0xb7, 0xe1, 0x62, 0x9f, 0xb2, 0x93, 0xf4, 0x78, 0xfe, 0x75, 0x1a, - 0xae, 0x44, 0xc5, 0x27, 0x0d, 0x71, 0xbe, 0xee, 0x0d, 0x5a, 0x83, 0xd9, 0x8e, 0x61, 0xbe, 0xd9, - 0xea, 0xe5, 0x4c, 0xc7, 0x30, 0x83, 0x65, 0xdd, 0x98, 0xae, 0x31, 0xf5, 0x53, 0xed, 0x1a, 0x2a, - 0x94, 0xe2, 0xda, 0x2e, 0xc9, 0xfe, 0xf1, 0x5d, 0x01, 0x66, 0x92, 0x5e, 0x96, 0x7b, 0xb3, 0xc0, - 0x3a, 0xa9, 0x09, 0xb3, 0x3f, 0x85, 0x75, 0xbc, 0xdf, 0x11, 0x00, 0x35, 0xed, 0x9e, 0x49, 0x40, - 0xed, 0x33, 0xab, 0x95, 0x64, 0x35, 0x2f, 0xc0, 0xa4, 0x61, 0xea, 0xf8, 0x88, 0x56, 0x33, 0x23, - 0xb3, 0x87, 0xc8, 0xd7, 0xc9, 0xf4, 0x58, 0x5f, 0x27, 0xa5, 0xcf, 0x61, 0x21, 0x52, 0xc4, 0x24, - 0xeb, 0xff, 0x3f, 0x52, 0xb0, 0xc0, 0x2b, 0x92, 0xf8, 0x0a, 0xe6, 0x37, 0x61, 0xb2, 0x4d, 0x64, - 0x8e, 0x68, 0x67, 0xfa, 0x4e, 0xaf, 0x9d, 0x69, 0x66, 0xf4, 0xb3, 0x00, 0x5d, 0x1b, 0x1f, 0x28, - 0x8c, 0x35, 0x3d, 0x16, 0x6b, 0x8e, 0x70, 0x50, 0x02, 0xfa, 0xbe, 0x00, 0x73, 0x64, 0x40, 0x77, - 0x6d, 0xab, 0x6b, 0x39, 0xc4, 0x67, 0x71, 0xc6, 0x83, 0x39, 0x2f, 0x4e, 0x4f, 0x16, 0x67, 0x37, - 0x0c, 0x73, 0x8b, 0x33, 0x36, 0x1b, 0x63, 0xef, 0x19, 0xf0, 0x76, 0x4e, 0x94, 0x2b, 0x6d, 0x4b, - 0xdb, 0x0f, 0xbe, 0xb7, 0x11, 0xcb, 0xe2, 0x8b, 0x73, 0xa4, 0x3f, 0x12, 0xe0, 0xc2, 0x4f, 0x6d, - 0xb9, 0xf8, 0x2f, 0x43, 0xd9, 0xd2, 0x4b, 0x10, 0xe9, 0x8f, 0xba, 0xb9, 0x6b, 0x25, 0xb9, 0x70, - 0xff, 0x6b, 0x02, 0xcc, 0x87, 0x04, 0x27, 0xe9, 0xe0, 0xbc, 0x91, 0x9e, 0xa4, 0x9f, 0x27, 0x2e, - 0x4f, 0x78, 0x94, 0x24, 0x39, 0x06, 0xff, 0x45, 0x0a, 0x2e, 0x55, 0xd8, 0x67, 0x6e, 0x2f, 0xee, - 0x23, 0xc9, 0x9e, 0x51, 0x84, 0xe9, 0x03, 0x6c, 0x3b, 0x86, 0xc5, 0xa6, 0xdf, 0x59, 0xd9, 0x7b, - 0x44, 0x25, 0xc8, 0x3a, 0xa6, 0xda, 0x75, 0xf6, 0x2c, 0xef, 0xfb, 0x9e, 0xff, 0xec, 0xc7, 0xa8, - 0x4c, 0xbe, 0x79, 0x8c, 0xca, 0xd4, 0xe8, 0x18, 0x95, 0xe9, 0xaf, 0x10, 0xa3, 0xc2, 0x3f, 0xa6, - 0xfd, 0x07, 0x01, 0x2e, 0x0f, 0x68, 0x2e, 0xc9, 0xde, 0xf2, 0x1d, 0xc8, 0x6b, 0x5c, 0x30, 0x31, - 0xd8, 0xec, 0x4b, 0x61, 0x9d, 0x64, 0x7b, 0x43, 0x2c, 0x72, 0x7a, 0xb2, 0x08, 0x5e, 0x51, 0xeb, - 0xab, 0x5c, 0x39, 0xe4, 0xb7, 0x2e, 0xfd, 0xf2, 0x2c, 0xcc, 0x55, 0x8f, 0xd8, 0x2a, 0x79, 0x83, - 0xb9, 0x09, 0xe8, 0x09, 0x64, 0xbb, 0xb6, 0x75, 0x60, 0x78, 0xd5, 0x28, 0x44, 0x02, 0x14, 0xbc, - 0x6a, 0xf4, 0x71, 0x6d, 0x71, 0x0e, 0xd9, 0xe7, 0x45, 0x4d, 0xc8, 0x3d, 0xb3, 0x34, 0xb5, 0xfd, - 0xc4, 0x68, 0x7b, 0x3d, 0xff, 0xfd, 0xb3, 0x05, 0x95, 0x7d, 0x9e, 0x2d, 0xd5, 0xdd, 0xf3, 0x1a, - 0xc1, 0x27, 0xa2, 0x3a, 0x64, 0x6b, 0xae, 0xdb, 0x25, 0x89, 0xdc, 0x76, 0xdc, 0x19, 0x43, 0x28, - 0x61, 0xf1, 0xa2, 0x6a, 0x3d, 0x76, 0xd4, 0x84, 0xf9, 0xa7, 0x74, 0x8f, 0x58, 0xa5, 0x6d, 0xf5, - 0xf4, 0x8a, 0x65, 0xee, 0x1a, 0x2d, 0x6e, 0xb7, 0x6f, 0x8f, 0x21, 0xf3, 0x69, 0xa5, 0x21, 0x0f, - 0x0a, 0x40, 0xcb, 0x90, 0x6d, 0x3c, 0xe2, 0xc2, 0x98, 0x5f, 0x77, 0x6b, 0x0c, 0x61, 0x8d, 0x47, - 0xb2, 0xcf, 0x86, 0xd6, 0x20, 0xbf, 0xfc, 0x45, 0xcf, 0xc6, 0x5c, 0xca, 0xd4, 0xd0, 0xe8, 0x88, - 0x7e, 0x29, 0x94, 0x4b, 0x0e, 0x33, 0xa3, 0x06, 0x14, 0x5e, 0x59, 0xf6, 0x7e, 0xdb, 0x52, 0xbd, - 0x1a, 0x4e, 0x53, 0x71, 0xdf, 0x18, 0x43, 0x9c, 0xc7, 0x28, 0xf7, 0x89, 0x40, 0xdf, 0x86, 0x39, - 0xd2, 0x18, 0x4d, 0x75, 0xa7, 0xed, 0x15, 0x32, 0x4b, 0xa5, 0xbe, 0x3b, 0x86, 0x54, 0x9f, 0xd3, - 0x5b, 0xf8, 0xef, 0x13, 0x55, 0x92, 0x61, 0x36, 0xd2, 0x09, 0x10, 0x82, 0x4c, 0x97, 0xb4, 0xb7, - 0x40, 0xe3, 0x97, 0xe8, 0x6f, 0xf4, 0x1e, 0x4c, 0x9b, 0x96, 0x8e, 0xbd, 0x11, 0x32, 0xbb, 0x72, - 0xe1, 0xf4, 0x64, 0x71, 0x6a, 0xd3, 0xd2, 0x99, 0x47, 0xc3, 0x7f, 0xc9, 0x53, 0x24, 0x53, 0x5d, - 0x2f, 0x5d, 0x83, 0x0c, 0x69, 0x77, 0x62, 0x98, 0x76, 0x54, 0x07, 0x6f, 0xdb, 0x06, 0x97, 0xe6, - 0x3d, 0x96, 0xfe, 0x71, 0x0a, 0x52, 0x8d, 0x47, 0xc4, 0x67, 0xdf, 0xe9, 0x69, 0xfb, 0xd8, 0xe5, - 0xe9, 0xfc, 0x89, 0xfa, 0xf2, 0x36, 0xde, 0x35, 0x98, 0x6b, 0x95, 0x93, 0xf9, 0x13, 0x7a, 0x07, - 0x40, 0xd5, 0x34, 0xec, 0x38, 0x8a, 0xb7, 0x77, 0x30, 0x27, 0xe7, 0x18, 0x65, 0x1d, 0x1f, 0x13, - 0x36, 0x07, 0x6b, 0x36, 0x76, 0xbd, 0xe0, 0x2b, 0xf6, 0x44, 0xd8, 0x5c, 0xdc, 0xe9, 0x2a, 0xae, - 0xb5, 0x8f, 0x4d, 0xda, 0x4f, 0x72, 0xc4, 0xd4, 0x74, 0xba, 0x4d, 0x42, 0x20, 0x56, 0x12, 0x9b, - 0x7a, 0x60, 0xd2, 0x72, 0xb2, 0xff, 0x4c, 0x44, 0xda, 0xb8, 0x65, 0xf0, 0x9d, 0x77, 0x39, 0x99, - 0x3f, 0x11, 0x2d, 0xa9, 0x3d, 0x77, 0x8f, 0xb6, 0x44, 0x4e, 0xa6, 0xbf, 0xd1, 0x6d, 0x98, 0x63, - 0xf1, 0x9a, 0x0a, 0x36, 0x35, 0x85, 0x1a, 0xd7, 0x1c, 0x4d, 0x9e, 0x65, 0xe4, 0xaa, 0xa9, 0x11, - 0x53, 0x8a, 0x1e, 0x01, 0x27, 0x28, 0xfb, 0x1d, 0x87, 0xe8, 0x14, 0x48, 0xae, 0x95, 0xb9, 0xd3, - 0x93, 0xc5, 0x7c, 0x83, 0x26, 0xac, 0x6f, 0x34, 0xea, 0xab, 0x72, 0x9e, 0xe5, 0x5a, 0xef, 0x38, - 0x75, 0xbd, 0xf4, 0xeb, 0x02, 0xa4, 0x9f, 0x56, 0x1a, 0xe7, 0x56, 0x99, 0x57, 0xd0, 0x74, 0xa8, - 0xa0, 0x77, 0x60, 0x6e, 0xc7, 0x68, 0xb7, 0x0d, 0xb3, 0x45, 0xbc, 0xa8, 0xef, 0x60, 0xcd, 0x53, - 0x58, 0x81, 0x93, 0xb7, 0x18, 0x15, 0x5d, 0x83, 0xbc, 0x66, 0x63, 0x1d, 0x9b, 0xae, 0xa1, 0xb6, - 0x1d, 0xae, 0xb9, 0x30, 0xa9, 0xf4, 0x4b, 0x02, 0x4c, 0xd2, 0x11, 0x80, 0xde, 0x86, 0x9c, 0x66, - 0x99, 0xae, 0x6a, 0x98, 0xdc, 0x94, 0xe5, 0xe4, 0x80, 0x30, 0xb4, 0x78, 0xd7, 0x61, 0x46, 0xd5, - 0x34, 0xab, 0x67, 0xba, 0x8a, 0xa9, 0x76, 0x30, 0x2f, 0x66, 0x9e, 0xd3, 0x36, 0xd5, 0x0e, 0x46, - 0x8b, 0xe0, 0x3d, 0xfa, 0x5b, 0x42, 0x73, 0x32, 0x70, 0xd2, 0x3a, 0x3e, 0x2e, 0xfd, 0x3b, 0x01, - 0xb2, 0xde, 0x98, 0x21, 0xc5, 0x68, 0x61, 0x93, 0x05, 0xa1, 0x7b, 0xc5, 0xf0, 0x09, 0xfd, 0x53, - 0x65, 0x2e, 0x98, 0x2a, 0x2f, 0xc0, 0xa4, 0x4b, 0x86, 0x05, 0x2f, 0x01, 0x7b, 0xa0, 0xeb, 0xd9, - 0x6d, 0xb5, 0xc5, 0x96, 0xf3, 0x72, 0x32, 0x7b, 0x20, 0x95, 0xe1, 0x61, 0xbf, 0x4c, 0x23, 0xfc, - 0x89, 0x94, 0x94, 0x05, 0xa7, 0xee, 0xe0, 0x96, 0x61, 0xd2, 0xbe, 0x94, 0x96, 0x81, 0x92, 0x56, - 0x08, 0x05, 0xbd, 0x05, 0x39, 0x96, 0x01, 0x9b, 0x3a, 0xed, 0x50, 0x69, 0x39, 0x4b, 0x09, 0x55, - 0x53, 0x2f, 0x61, 0xc8, 0xf9, 0x83, 0x93, 0x34, 0x5b, 0xcf, 0xf1, 0x15, 0x49, 0x7f, 0xa3, 0xf7, - 0xe1, 0xc2, 0xeb, 0x9e, 0xda, 0x36, 0x76, 0xe9, 0x4a, 0x1d, 0x8d, 0xd2, 0xa7, 0x3a, 0x63, 0x35, - 0x41, 0x7e, 0x1a, 0x95, 0x40, 0x55, 0xe7, 0x8d, 0xe5, 0x74, 0x30, 0x96, 0xa5, 0xdf, 0x13, 0x60, - 0x9e, 0x85, 0x51, 0xb1, 0x80, 0xd9, 0xe4, 0xfc, 0x90, 0x8f, 0x21, 0xa7, 0xab, 0xae, 0xca, 0x36, - 0xb9, 0xa6, 0x46, 0x6e, 0x72, 0xf5, 0x37, 0x5d, 0xa8, 0xae, 0x4a, 0x37, 0xba, 0x22, 0xc8, 0x90, - 0xdf, 0x6c, 0x3f, 0xb0, 0x4c, 0x7f, 0x4b, 0x9f, 0x01, 0x0a, 0x17, 0x34, 0x49, 0x8f, 0xec, 0x1e, - 0x5c, 0x24, 0xba, 0xae, 0x9a, 0x9a, 0x7d, 0xdc, 0x75, 0x0d, 0xcb, 0x7c, 0x4e, 0xff, 0x3a, 0x48, - 0x0c, 0x7d, 0xd8, 0xa2, 0xdf, 0xb3, 0xa4, 0xdf, 0x9f, 0x82, 0xd9, 0xea, 0x51, 0xd7, 0xb2, 0x13, - 0x5d, 0x06, 0x5b, 0x81, 0x69, 0xbe, 0x52, 0x30, 0xe2, 0xdb, 0x75, 0x9f, 0x31, 0xf7, 0x3e, 0xdc, - 0x73, 0x46, 0xb4, 0x02, 0xc0, 0x82, 0x5a, 0x69, 0xe0, 0x52, 0xfa, 0x1c, 0x9f, 0xda, 0x28, 0x1b, - 0xdd, 0xf0, 0xb1, 0x09, 0xf9, 0xce, 0x81, 0xa6, 0x29, 0xbb, 0x46, 0xdb, 0xe5, 0xb1, 0x81, 0xf1, - 0x61, 0xec, 0x1b, 0x2f, 0x2b, 0x95, 0x27, 0x34, 0x13, 0x8b, 0xa9, 0x0b, 0x9e, 0x65, 0x20, 0x12, - 0xd8, 0x6f, 0xf4, 0x2e, 0xf0, 0xcd, 0x47, 0x8a, 0xe3, 0x6d, 0x25, 0x5c, 0x99, 0x3d, 0x3d, 0x59, - 0xcc, 0xc9, 0x94, 0xda, 0x68, 0x34, 0xe5, 0x1c, 0xcb, 0xd0, 0x70, 0x5c, 0x74, 0x03, 0x66, 0xad, - 0x8e, 0xe1, 0x2a, 0x9e, 0x93, 0xc4, 0x3d, 0xca, 0x19, 0x42, 0xf4, 0x9c, 0xa8, 0xf3, 0xec, 0x49, - 0x99, 0x1e, 0x7f, 0x4f, 0xca, 0xdf, 0x16, 0xe0, 0x12, 0x57, 0xa4, 0xb2, 0x43, 0xe3, 0xf0, 0xd5, - 0xb6, 0xe1, 0x1e, 0x2b, 0xfb, 0x07, 0xc5, 0x2c, 0xf5, 0x5b, 0x7f, 0x26, 0xb6, 0x41, 0x42, 0xfd, - 0xa0, 0xec, 0x35, 0xcb, 0xf1, 0x33, 0xce, 0xbc, 0x7e, 0x50, 0x35, 0x5d, 0xfb, 0x78, 0xe5, 0xf2, - 0xe9, 0xc9, 0xe2, 0xc2, 0x60, 0xea, 0x4b, 0x79, 0xc1, 0x19, 0x64, 0x41, 0x35, 0x00, 0xec, 0xf7, - 0x43, 0x3a, 0x63, 0xc4, 0xfb, 0x1f, 0xb1, 0x1d, 0x56, 0x0e, 0xf1, 0xa2, 0xbb, 0x20, 0xf2, 0x3d, - 0x40, 0xbb, 0x46, 0x1b, 0x2b, 0x8e, 0xf1, 0x05, 0xa6, 0x73, 0x4b, 0x5a, 0x2e, 0x30, 0x3a, 0x11, - 0xd1, 0x30, 0xbe, 0xc0, 0xa5, 0xef, 0x40, 0x71, 0x58, 0xe9, 0xc3, 0x43, 0x20, 0xc7, 0x3e, 0xe9, - 0x7e, 0x14, 0x5d, 0xcf, 0x19, 0xa3, 0xab, 0xf2, 0x35, 0x9d, 0x8f, 0x53, 0x1f, 0x09, 0xd2, 0x3f, - 0x4c, 0xc1, 0xec, 0x4a, 0xaf, 0xbd, 0xff, 0xbc, 0xdb, 0x60, 0x87, 0x21, 0x10, 0x33, 0xc8, 0x0c, - 0x05, 0x29, 0xa0, 0xc0, 0xcc, 0x20, 0xb5, 0x04, 0xc6, 0x17, 0x98, 0x4c, 0x4e, 0xa1, 0x70, 0x19, - 0xbe, 0xcf, 0x80, 0xd6, 0x21, 0x20, 0xd3, 0xad, 0x00, 0x1f, 0x41, 0x31, 0x94, 0x91, 0x2e, 0xbe, - 0x28, 0xd8, 0x74, 0x6d, 0x03, 0xb3, 0x05, 0xc4, 0xb4, 0x1c, 0x8a, 0xe9, 0xa9, 0x93, 0xe4, 0x2a, - 0x4b, 0x45, 0x4d, 0x98, 0x21, 0x19, 0x8f, 0x15, 0x3a, 0x85, 0x78, 0x0b, 0xbc, 0x0f, 0x62, 0xaa, - 0x15, 0x29, 0x77, 0x99, 0xea, 0xa7, 0x42, 0x79, 0xe8, 0x4f, 0x39, 0x8f, 0x03, 0x4a, 0xe9, 0x53, - 0x10, 0xfb, 0x33, 0x84, 0x75, 0x99, 0x61, 0xba, 0xbc, 0x10, 0xd6, 0x65, 0x3a, 0xa4, 0xa7, 0xb5, - 0x4c, 0x36, 0x23, 0x4e, 0x4a, 0x7f, 0x96, 0x86, 0x82, 0xd7, 0xcd, 0x92, 0x04, 0x3a, 0x2b, 0x30, - 0x49, 0x3a, 0x85, 0x17, 0x81, 0x72, 0x7b, 0x44, 0xef, 0xe6, 0x91, 0xed, 0xa4, 0xb3, 0x78, 0x20, - 0x99, 0xb2, 0x26, 0x61, 0x70, 0x4a, 0xbf, 0x94, 0x82, 0x0c, 0xc5, 0x16, 0x0f, 0x20, 0x43, 0x27, - 0x0a, 0x61, 0x9c, 0x89, 0x82, 0x66, 0xf5, 0xa7, 0xb3, 0x54, 0xc8, 0x35, 0x25, 0x3e, 0xdf, 0x9e, - 0xfa, 0xc1, 0x83, 0x87, 0xd4, 0xd8, 0xcc, 0xc8, 0xfc, 0x09, 0xad, 0xd0, 0xd0, 0x28, 0xcb, 0x76, - 0xb1, 0xce, 0x7d, 0xfa, 0x6b, 0x67, 0xb5, 0xaf, 0x37, 0x29, 0x79, 0x7c, 0xe8, 0x0a, 0xa4, 0x89, - 0x15, 0x9b, 0x66, 0x51, 0x0e, 0xa7, 0x27, 0x8b, 0x69, 0x62, 0xbf, 0x08, 0x0d, 0x2d, 0x41, 0x3e, - 0x6a, 0x32, 0x88, 0x07, 0x47, 0x0d, 0x63, 0x68, 0xb8, 0x43, 0xdb, 0x1f, 0x5a, 0x0c, 0xcf, 0xf2, - 0x36, 0xfe, 0x5f, 0x19, 0x98, 0xad, 0x77, 0x92, 0x9e, 0x52, 0x96, 0xa3, 0x2d, 0x1c, 0x07, 0x84, - 0x22, 0x2f, 0x8d, 0x69, 0xe0, 0xc8, 0x0c, 0x9e, 0x3e, 0xdf, 0x0c, 0xfe, 0x29, 0xf5, 0xa2, 0x59, - 0xd7, 0x98, 0x1a, 0xbf, 0x6b, 0x4c, 0x63, 0x53, 0xa7, 0x33, 0x51, 0x9d, 0x78, 0xda, 0xfc, 0x4c, - 0x8a, 0xf4, 0x10, 0xcc, 0x14, 0x2d, 0x3f, 0xf5, 0x73, 0x64, 0xc2, 0x13, 0xec, 0x15, 0xa1, 0x91, - 0x2e, 0x51, 0x8b, 0x3a, 0xfd, 0xe6, 0x16, 0xb5, 0xe4, 0xf2, 0xce, 0xfa, 0x31, 0xa4, 0x75, 0xc3, - 0x6b, 0x9c, 0xf1, 0xa7, 0x6a, 0xc2, 0x74, 0x46, 0xaf, 0xcd, 0x84, 0x7b, 0x2d, 0xeb, 0x25, 0xa5, - 0x3a, 0x40, 0x50, 0x37, 0x74, 0x0d, 0xa6, 0xac, 0xb6, 0xee, 0x6d, 0x76, 0x99, 0x5d, 0xc9, 0x9d, - 0x9e, 0x2c, 0x4e, 0x3e, 0x6f, 0xeb, 0xf5, 0x55, 0x79, 0xd2, 0x6a, 0xeb, 0x75, 0x9d, 0x1e, 0xe8, - 0x81, 0x0f, 0x15, 0x3f, 0x12, 0x6e, 0x46, 0x9e, 0x36, 0xf1, 0xe1, 0x2a, 0x76, 0x34, 0xde, 0xe1, - 0x7e, 0x5b, 0x80, 0x82, 0xa7, 0xbb, 0x64, 0x8d, 0x4a, 0xd6, 0xe8, 0xf0, 0x41, 0x96, 0x3e, 0xdf, - 0x20, 0xf3, 0xf8, 0xf8, 0x46, 0xe4, 0x5f, 0x11, 0x78, 0x6c, 0x73, 0x43, 0x53, 0x5d, 0xe2, 0x54, - 0x24, 0x38, 0x30, 0xee, 0x81, 0x68, 0xab, 0xa6, 0x6e, 0x75, 0x8c, 0x2f, 0x30, 0x5b, 0x08, 0x75, - 0xf8, 0x67, 0xcf, 0x39, 0x9f, 0x4e, 0x57, 0xfd, 0x1c, 0xe9, 0xbf, 0x0b, 0x3c, 0x0e, 0xda, 0x2f, - 0x46, 0xb2, 0xc1, 0x29, 0x79, 0xfe, 0x69, 0xc0, 0xdc, 0xb5, 0xbc, 0x30, 0xae, 0xb7, 0x87, 0x05, - 0x2d, 0xd6, 0xcd, 0x5d, 0xcb, 0xfb, 0xcc, 0x6e, 0x7b, 0x04, 0xa7, 0xf4, 0x73, 0x30, 0x49, 0x93, - 0xdf, 0xc0, 0x80, 0xfa, 0xb1, 0xf7, 0x44, 0xe3, 0x7f, 0x9a, 0x82, 0x9b, 0xb4, 0xaa, 0x2f, 0xb1, - 0x6d, 0xec, 0x1e, 0x6f, 0xd9, 0x96, 0x8b, 0x35, 0x17, 0xeb, 0xc1, 0xea, 0x76, 0x82, 0x4d, 0xa0, - 0x43, 0x8e, 0xc7, 0x05, 0x18, 0x3a, 0x3f, 0xd4, 0xe7, 0xe9, 0x57, 0x5b, 0x64, 0xcb, 0xb2, 0x78, - 0x82, 0xfa, 0xaa, 0x9c, 0x65, 0x92, 0xeb, 0x3a, 0x5a, 0x86, 0x5c, 0xd7, 0xab, 0xc6, 0xb9, 0x42, - 0xcf, 0x7c, 0x2e, 0xb4, 0x0e, 0x73, 0xbc, 0xa0, 0x6a, 0xdb, 0x38, 0xc0, 0x8a, 0xea, 0x9e, 0x67, - 0x9e, 0x9b, 0x65, 0xbc, 0xcb, 0x84, 0x75, 0xd9, 0x95, 0xfe, 0x4e, 0x06, 0x6e, 0x9d, 0xa1, 0xe2, - 0x24, 0xbb, 0x57, 0x09, 0xb2, 0x07, 0xe4, 0x45, 0x06, 0xaf, 0x7d, 0x56, 0xf6, 0x9f, 0xd1, 0x4e, - 0xc4, 0x59, 0xda, 0x55, 0x8d, 0x36, 0x71, 0xae, 0x58, 0xb0, 0xef, 0xf0, 0x70, 0xc2, 0xf8, 0xe0, - 0xd9, 0x90, 0x5b, 0xf5, 0x84, 0x0a, 0xa2, 0xd9, 0x1c, 0xf4, 0x5d, 0x01, 0x4a, 0xec, 0x85, 0x2c, - 0xe2, 0xb4, 0xef, 0x35, 0x19, 0xfa, 0x9a, 0xd5, 0x98, 0xd7, 0x8c, 0xa5, 0xa3, 0x72, 0xe8, 0x5d, - 0xbc, 0x20, 0xc5, 0xf0, 0xdb, 0xc2, 0x45, 0x29, 0xfd, 0x86, 0x00, 0xf9, 0x10, 0x01, 0xdd, 0x1e, - 0xd8, 0x32, 0x98, 0x3f, 0x8d, 0xdb, 0x27, 0x78, 0x6b, 0x60, 0x9f, 0xe0, 0x4a, 0xf6, 0xcb, 0x93, - 0xc5, 0x8c, 0xcc, 0xf6, 0x8d, 0x78, 0x3b, 0x06, 0xaf, 0x07, 0x27, 0x54, 0xa5, 0xfb, 0x32, 0x79, - 0x47, 0x54, 0xd1, 0x85, 0x23, 0xd5, 0xfb, 0x1c, 0x4d, 0x17, 0x8e, 0xc8, 0x93, 0xf4, 0x9b, 0x29, - 0x98, 0x5f, 0xd6, 0xf5, 0x46, 0x83, 0x5b, 0xf8, 0xe4, 0xc6, 0x98, 0x07, 0xa1, 0x53, 0x01, 0x84, - 0x46, 0xef, 0x01, 0xd2, 0x0d, 0x87, 0x9d, 0xf4, 0xe2, 0xec, 0xa9, 0xba, 0x75, 0x18, 0x44, 0x9d, - 0xcc, 0x7b, 0x29, 0x0d, 0x2f, 0x01, 0x35, 0x80, 0x62, 0x39, 0xc5, 0x71, 0x55, 0xff, 0xab, 0xda, - 0xad, 0xb1, 0x36, 0xcc, 0x31, 0x90, 0xe7, 0x3f, 0xca, 0x39, 0x22, 0x87, 0xfe, 0x24, 0xa8, 0xc4, - 0x20, 0x8d, 0xe2, 0x2a, 0xaa, 0xe3, 0x6d, 0xf5, 0x62, 0x67, 0xcc, 0x14, 0x18, 0x7d, 0xd9, 0x61, - 0x3b, 0xb8, 0xd8, 0x76, 0x8e, 0x40, 0x35, 0x49, 0x02, 0xfe, 0xbf, 0x2f, 0x40, 0x41, 0xc6, 0xbb, - 0x36, 0x76, 0x12, 0x5d, 0xf2, 0x78, 0x02, 0x33, 0x36, 0x93, 0xaa, 0xec, 0xda, 0x56, 0xe7, 0x3c, - 0xb6, 0x22, 0xcf, 0x19, 0x9f, 0xd8, 0x56, 0x87, 0x9b, 0xe4, 0x97, 0x30, 0xe7, 0x97, 0x31, 0xc9, - 0xca, 0xff, 0x1e, 0xdd, 0x0c, 0xce, 0x04, 0x27, 0x1d, 0xfe, 0x91, 0xac, 0x06, 0xe8, 0x67, 0xb8, - 0x70, 0x41, 0x93, 0x54, 0xc3, 0x7f, 0x13, 0xa0, 0xd0, 0xe8, 0xed, 0xb0, 0x13, 0xcc, 0x92, 0xd3, - 0x40, 0x15, 0x72, 0x6d, 0xbc, 0xeb, 0x2a, 0x6f, 0xb4, 0x11, 0x21, 0x4b, 0x58, 0xe9, 0x36, 0x8c, - 0xa7, 0x00, 0x36, 0xdd, 0x0d, 0x49, 0xe5, 0xa4, 0xcf, 0x29, 0x27, 0x47, 0x79, 0x09, 0x99, 0xcc, - 0x3a, 0x73, 0x7e, 0x35, 0x93, 0x9c, 0x5f, 0x5e, 0x45, 0xac, 0x43, 0xfa, 0x3c, 0xd6, 0x61, 0x9e, - 0x47, 0xbc, 0xc4, 0x5b, 0x88, 0x32, 0x2c, 0x50, 0xb7, 0x4c, 0x51, 0xbb, 0xdd, 0xb6, 0xe1, 0x81, - 0x79, 0x6a, 0x7f, 0x32, 0xf2, 0x3c, 0x4d, 0x5a, 0x66, 0x29, 0x14, 0xc6, 0xa3, 0x5f, 0x15, 0x60, - 0x66, 0xd7, 0xc6, 0xf8, 0x0b, 0xac, 0x50, 0x93, 0x3c, 0x5e, 0x48, 0xcf, 0x2a, 0x29, 0xc3, 0x57, - 0xfe, 0xe4, 0x9f, 0x67, 0x2f, 0x6e, 0x90, 0xf7, 0xa2, 0x4d, 0x10, 0xb5, 0x36, 0x0b, 0x42, 0xf0, - 0xc3, 0x8b, 0xce, 0x81, 0x7d, 0xe6, 0x18, 0x73, 0x10, 0x61, 0xf4, 0x82, 0x0c, 0x26, 0x55, 0x57, - 0xf8, 0xa9, 0x91, 0x1c, 0xba, 0x94, 0x87, 0x9c, 0x0a, 0x11, 0x3a, 0x6c, 0xb2, 0x2c, 0x63, 0x55, - 0xe7, 0x1e, 0x36, 0x19, 0x57, 0xfe, 0x03, 0x1f, 0x57, 0xaf, 0x60, 0x9e, 0xf6, 0x9b, 0xa4, 0x37, - 0x77, 0x4b, 0x3f, 0x4c, 0x01, 0x0a, 0x4b, 0xfe, 0xe9, 0xf5, 0xb7, 0x54, 0x72, 0xfd, 0xed, 0x5d, - 0x40, 0x2c, 0x6e, 0xd5, 0x51, 0xba, 0xd8, 0x56, 0x1c, 0xac, 0x59, 0xfc, 0x3c, 0x2f, 0x41, 0x16, - 0x79, 0xca, 0x16, 0xb6, 0x1b, 0x94, 0x8e, 0x96, 0x01, 0x02, 0xaf, 0x9d, 0x4f, 0x8a, 0xe3, 0x38, - 0xed, 0x39, 0xdf, 0x69, 0x97, 0xbe, 0x27, 0x40, 0x61, 0xc3, 0x68, 0xd9, 0x6a, 0xa2, 0xc7, 0x55, - 0xa1, 0x8f, 0xa3, 0x5f, 0x33, 0xf2, 0x0f, 0x4b, 0x71, 0x91, 0x56, 0x2c, 0x87, 0x07, 0xb7, 0x39, - 0x03, 0x99, 0x6b, 0xfc, 0x12, 0x25, 0x69, 0x64, 0xff, 0x63, 0x09, 0x66, 0x78, 0xb9, 0xb7, 0x4d, - 0xc3, 0x32, 0xd1, 0x03, 0x48, 0xb7, 0xf8, 0xd7, 0xaa, 0x7c, 0xec, 0xca, 0x72, 0x70, 0x18, 0x64, - 0x6d, 0x42, 0x26, 0x79, 0x09, 0x4b, 0xb7, 0xe7, 0xc6, 0x78, 0xf0, 0xc1, 0xe6, 0x83, 0x30, 0x4b, - 0xb7, 0xe7, 0xa2, 0x06, 0xcc, 0x69, 0xc1, 0x09, 0x74, 0x0a, 0x61, 0x4f, 0x0f, 0xc5, 0xfd, 0xb1, - 0x67, 0x01, 0xd6, 0x26, 0xe4, 0x82, 0x16, 0x49, 0x40, 0x95, 0xf0, 0xc1, 0x67, 0x99, 0x81, 0xc8, - 0xc6, 0x60, 0xdb, 0x7c, 0xf4, 0xd0, 0xb5, 0xda, 0x44, 0xe8, 0x7c, 0x34, 0xf4, 0x31, 0x4c, 0xe9, - 0xf4, 0x88, 0x2d, 0x6e, 0xa5, 0xe2, 0x1a, 0x3a, 0x72, 0x92, 0x59, 0x6d, 0x42, 0xe6, 0x1c, 0x68, - 0x0d, 0x66, 0xd8, 0x2f, 0xe6, 0x43, 0x73, 0xdb, 0x72, 0x6b, 0xb8, 0x84, 0xd0, 0xec, 0x5e, 0x9b, - 0x90, 0xf3, 0x7a, 0x40, 0x45, 0x4f, 0x21, 0xaf, 0xb5, 0xb1, 0x6a, 0x73, 0x51, 0xb7, 0x87, 0x6e, - 0xc7, 0x1c, 0x38, 0x96, 0xab, 0x36, 0x21, 0x83, 0xe6, 0x13, 0x49, 0xa1, 0x6c, 0x7a, 0x3a, 0x13, - 0x97, 0xf4, 0xfe, 0xd0, 0x42, 0x0d, 0x1e, 0x75, 0x55, 0xa3, 0xb3, 0xbe, 0x4f, 0x45, 0xdf, 0x84, - 0x8c, 0xa3, 0xa9, 0xde, 0x1a, 0xcd, 0xd5, 0x21, 0xc7, 0xe7, 0x04, 0xcc, 0x34, 0x37, 0x7a, 0xcc, - 0xdc, 0x6f, 0xf7, 0xc8, 0x5b, 0x2e, 0x8f, 0xd3, 0x69, 0xe4, 0x98, 0x06, 0xa2, 0x53, 0x4c, 0x09, - 0x44, 0x0f, 0x2a, 0xc1, 0x1b, 0x0a, 0xdd, 0xe8, 0x4c, 0xd7, 0xc7, 0xe3, 0xf5, 0x30, 0xb0, 0x31, - 0xbd, 0x46, 0xcf, 0x82, 0xf0, 0x88, 0x68, 0x03, 0x66, 0x99, 0xa0, 0x1e, 0xdb, 0x33, 0x5d, 0x5c, - 0x1a, 0x1a, 0xcd, 0x10, 0xb3, 0x6b, 0xbb, 0x36, 0x21, 0xcf, 0xa8, 0x21, 0x72, 0x50, 0xae, 0x0e, - 0xb6, 0x5b, 0xb8, 0x98, 0x1f, 0x5d, 0xae, 0x70, 0xcc, 0xa6, 0x5f, 0x2e, 0x4a, 0x44, 0xbf, 0x08, - 0x17, 0x98, 0x20, 0x97, 0x87, 0xa2, 0xf1, 0xe0, 0xaf, 0x77, 0x86, 0x06, 0x0d, 0x0c, 0xdd, 0xe7, - 0x5c, 0x9b, 0x90, 0x91, 0x3a, 0x90, 0x88, 0x34, 0xb8, 0xc8, 0xde, 0xc0, 0x37, 0xca, 0xda, 0x7c, - 0x6f, 0x67, 0xf1, 0x06, 0x7d, 0xc5, 0x7b, 0xc3, 0x5e, 0x11, 0xbb, 0x7f, 0xb7, 0x36, 0x21, 0x2f, - 0xa8, 0x83, 0xa9, 0x41, 0x35, 0x6c, 0xbe, 0x25, 0x91, 0x77, 0xb7, 0xf7, 0x46, 0x57, 0x23, 0x6e, - 0x2b, 0xa7, 0x5f, 0x8d, 0x48, 0x22, 0x69, 0x40, 0xff, 0x28, 0x06, 0xda, 0x99, 0x66, 0x86, 0x36, - 0x60, 0xcc, 0xbe, 0x45, 0xd2, 0x80, 0x7b, 0x21, 0x32, 0x2a, 0x43, 0xaa, 0xa5, 0x15, 0x67, 0x87, - 0xce, 0x0f, 0xfe, 0xde, 0xbc, 0xda, 0x84, 0x9c, 0x6a, 0x69, 0xe8, 0x53, 0xc8, 0xb2, 0x8d, 0x56, - 0x47, 0x66, 0xb1, 0x30, 0xd4, 0xe0, 0x46, 0xb7, 0xab, 0xd5, 0x26, 0x64, 0xba, 0xb7, 0x8b, 0x77, - 0x64, 0xbe, 0x89, 0x86, 0x8a, 0x28, 0x8f, 0xd8, 0x5f, 0xdd, 0xb7, 0x95, 0x89, 0x74, 0x18, 0xdb, - 0x27, 0xa2, 0x2d, 0x28, 0xd8, 0x2c, 0xcc, 0xd8, 0xdb, 0x14, 0x20, 0x0e, 0x8d, 0xf5, 0x89, 0xdb, - 0x17, 0x50, 0xa3, 0x0b, 0x1f, 0x21, 0x3a, 0x69, 0xbb, 0xa8, 0x44, 0xde, 0x76, 0xf3, 0x43, 0xdb, - 0x6e, 0x68, 0x8c, 0x3a, 0x69, 0x3b, 0x7b, 0x20, 0x11, 0x7d, 0x08, 0x93, 0x6c, 0x9c, 0x20, 0x2a, - 0x32, 0x2e, 0x7c, 0xad, 0x6f, 0x88, 0xb0, 0xfc, 0xc4, 0x7a, 0xb9, 0x3c, 0xd6, 0x56, 0x69, 0x5b, - 0xad, 0xe2, 0xc2, 0x50, 0xeb, 0x35, 0x18, 0x35, 0x4c, 0xac, 0x97, 0x1b, 0x50, 0x49, 0x07, 0xb2, - 0x59, 0x0a, 0x1f, 0x62, 0x17, 0x86, 0x76, 0xa0, 0x98, 0x10, 0xdc, 0x1a, 0xdd, 0x05, 0x15, 0x90, - 0x7d, 0xc3, 0xea, 0x60, 0x85, 0x1a, 0xc5, 0x8b, 0xa3, 0x0d, 0x6b, 0xe4, 0x4c, 0x33, 0xdf, 0xb0, - 0x32, 0x2a, 0x7a, 0x09, 0x22, 0x3f, 0x58, 0x27, 0xf8, 0xb8, 0x7a, 0x89, 0xca, 0xbb, 0x17, 0x3b, - 0x21, 0xc6, 0x05, 0x27, 0xd6, 0x88, 0x87, 0x1a, 0x4d, 0x41, 0x9f, 0xc1, 0x3c, 0x95, 0xa7, 0x68, - 0xc1, 0x59, 0x48, 0xc5, 0xe2, 0xc0, 0xc9, 0x3a, 0xc3, 0x8f, 0x4d, 0xf2, 0x24, 0x8b, 0x5a, 0x5f, - 0x12, 0x19, 0x0f, 0x86, 0x69, 0xb8, 0x74, 0xee, 0x2e, 0x0d, 0x1d, 0x0f, 0xd1, 0x73, 0x60, 0xc9, - 0x78, 0x30, 0x18, 0x85, 0x74, 0xe3, 0x3e, 0x8b, 0xf7, 0xf6, 0xd0, 0x6e, 0x3c, 0xc4, 0xd8, 0xcd, - 0xba, 0x11, 0x3b, 0xb7, 0x0a, 0xc0, 0x70, 0x09, 0xf5, 0xfc, 0xae, 0x0e, 0x75, 0x00, 0xfa, 0x43, - 0x64, 0x89, 0x03, 0xd0, 0xf6, 0x68, 0x64, 0x9c, 0xd2, 0x55, 0x0f, 0x85, 0x46, 0x7a, 0x14, 0x17, - 0x87, 0x8e, 0xd3, 0x81, 0xa8, 0x0c, 0x32, 0x4e, 0x0f, 0x7d, 0x22, 0xf1, 0x24, 0xd8, 0xa7, 0xa9, - 0xe2, 0xb5, 0xe1, 0xb3, 0x5e, 0xf8, 0x0b, 0x35, 0x9d, 0xf5, 0x28, 0x81, 0xf0, 0xb2, 0x15, 0xf7, - 0xa2, 0x34, 0x94, 0x37, 0xf2, 0x75, 0x85, 0xf0, 0x32, 0x0e, 0xb4, 0x0c, 0x39, 0xe2, 0x14, 0x1f, - 0x53, 0x33, 0x73, 0x7d, 0x28, 0x30, 0xed, 0xdb, 0xc0, 0x57, 0x9b, 0x90, 0xb3, 0xaf, 0x39, 0x89, - 0x74, 0x6d, 0x26, 0x82, 0x1b, 0x98, 0xfb, 0x43, 0xbb, 0xf6, 0xe0, 0xce, 0x2d, 0xd2, 0xb5, 0x5f, - 0x07, 0xd4, 0x60, 0xde, 0x75, 0xd8, 0x1a, 0x7d, 0xf1, 0xe6, 0xe8, 0x79, 0x37, 0xfa, 0x45, 0xc1, - 0x9f, 0x77, 0x39, 0x99, 0xcd, 0xbb, 0xba, 0xe2, 0x38, 0x2c, 0xfc, 0xe7, 0xd6, 0x88, 0x79, 0xb7, - 0x6f, 0xd5, 0x8e, 0xcd, 0xbb, 0x7a, 0x83, 0x71, 0x12, 0x17, 0xd4, 0xf6, 0x4e, 0xb3, 0xe2, 0x98, - 0xe5, 0xce, 0x50, 0x17, 0x34, 0xf6, 0xb8, 0x2d, 0xe2, 0x82, 0xda, 0x91, 0x04, 0xf4, 0xb3, 0x30, - 0xcd, 0x57, 0x49, 0x8a, 0x77, 0x47, 0x38, 0xe5, 0xe1, 0x85, 0x2d, 0x32, 0x26, 0x38, 0x0f, 0xb3, - 0x50, 0x6c, 0x75, 0x86, 0x59, 0xe0, 0x7b, 0x23, 0x2c, 0xd4, 0xc0, 0x02, 0x11, 0xb3, 0x50, 0x01, - 0x99, 0x94, 0xc6, 0x61, 0x2b, 0x0b, 0xc5, 0x6f, 0x0c, 0x2d, 0x4d, 0x74, 0x89, 0x85, 0x94, 0x86, - 0xf3, 0xd0, 0x19, 0x8b, 0x3a, 0x0c, 0x4c, 0x3b, 0xef, 0x0e, 0x9f, 0xb1, 0xfa, 0xb1, 0x6a, 0xcd, - 0xfb, 0x06, 0xc2, 0xb4, 0xf2, 0xb7, 0x04, 0xb8, 0xc6, 0xfa, 0x00, 0x5d, 0x01, 0x3e, 0x56, 0xfc, - 0x05, 0xfc, 0x10, 0x10, 0x7f, 0x40, 0xc5, 0x7f, 0x78, 0xfe, 0xf5, 0x66, 0xef, 0x8d, 0xef, 0xa8, - 0xa3, 0xf2, 0x11, 0x65, 0x74, 0x18, 0x82, 0x2a, 0x3e, 0x1c, 0xaa, 0x8c, 0x28, 0xea, 0x23, 0xca, - 0xe0, 0x3c, 0x2b, 0xd3, 0x3c, 0x10, 0xc0, 0xdf, 0x17, 0x3d, 0x27, 0x8a, 0x6b, 0x99, 0xec, 0x65, - 0xb1, 0xb8, 0x96, 0xc9, 0x5e, 0x11, 0x4b, 0x6b, 0x99, 0xec, 0x5b, 0xe2, 0xdb, 0xd2, 0x3f, 0x2a, - 0xc1, 0xac, 0x07, 0xba, 0x18, 0xa0, 0x7a, 0x18, 0x06, 0x54, 0x57, 0x87, 0x01, 0x2a, 0x0e, 0xd3, - 0x38, 0xa2, 0x7a, 0x18, 0x46, 0x54, 0x57, 0x87, 0x21, 0xaa, 0x80, 0x87, 0x40, 0xaa, 0xe6, 0x30, - 0x48, 0x75, 0x6f, 0x0c, 0x48, 0xe5, 0x8b, 0xea, 0xc7, 0x54, 0xab, 0x83, 0x98, 0xea, 0xe6, 0x68, - 0x4c, 0xe5, 0x8b, 0x0a, 0x81, 0xaa, 0xc7, 0x7d, 0xa0, 0xea, 0xfa, 0x08, 0x50, 0xe5, 0xf3, 0x7b, - 0xa8, 0x6a, 0x3d, 0x16, 0x55, 0xdd, 0x3e, 0x0b, 0x55, 0xf9, 0x72, 0x22, 0xb0, 0xaa, 0x16, 0x07, - 0xab, 0x6e, 0x9d, 0x01, 0xab, 0x7c, 0x51, 0x61, 0x5c, 0xb5, 0x1e, 0x8b, 0xab, 0x6e, 0x9f, 0x85, - 0xab, 0x82, 0x62, 0x85, 0x81, 0xd5, 0x07, 0x11, 0x60, 0xb5, 0x38, 0x14, 0x58, 0xf9, 0xdc, 0x0c, - 0x59, 0x7d, 0xd2, 0x8f, 0xac, 0xae, 0x8f, 0x40, 0x56, 0x81, 0x62, 0x39, 0xb4, 0xaa, 0xc5, 0x41, - 0xab, 0x5b, 0x67, 0x40, 0xab, 0x40, 0x17, 0x21, 0x6c, 0xb5, 0x19, 0x8f, 0xad, 0xee, 0x9c, 0x89, - 0xad, 0x7c, 0x69, 0x51, 0x70, 0x55, 0x8b, 0x03, 0x57, 0xb7, 0xce, 0x00, 0x57, 0x7d, 0x25, 0x63, - 0xe8, 0x4a, 0x1d, 0x89, 0xae, 0xde, 0x1b, 0x13, 0x5d, 0xf9, 0xa2, 0xe3, 0xe0, 0x95, 0x3e, 0x1a, - 0x5e, 0x95, 0xc7, 0x85, 0x57, 0xfe, 0x4b, 0x62, 0xf1, 0x95, 0x3a, 0x12, 0x5f, 0xbd, 0x37, 0x26, - 0xbe, 0xea, 0xab, 0x48, 0x14, 0x60, 0x6d, 0xc6, 0x03, 0xac, 0x3b, 0x67, 0x02, 0xac, 0xa0, 0x15, - 0x23, 0x08, 0x6b, 0x29, 0x84, 0xb0, 0xde, 0x19, 0x82, 0xb0, 0x7c, 0x56, 0x02, 0xb1, 0xbe, 0x35, - 0x00, 0xb1, 0xa4, 0x51, 0x10, 0xcb, 0xe7, 0xf5, 0x31, 0x56, 0x2d, 0x0e, 0x63, 0xdd, 0x3a, 0x03, - 0x63, 0x05, 0xfd, 0x26, 0x04, 0xb2, 0x5e, 0x0c, 0x01, 0x59, 0x77, 0xcf, 0x06, 0x59, 0xbe, 0xbc, - 0x3e, 0x94, 0xa5, 0x8e, 0x44, 0x59, 0xef, 0x8d, 0x89, 0xb2, 0x82, 0x16, 0x8c, 0x81, 0x59, 0x1f, - 0x45, 0x61, 0xd6, 0xb5, 0xe1, 0x30, 0xcb, 0x17, 0xc3, 0x71, 0xd6, 0x7a, 0x2c, 0xce, 0xba, 0x7d, - 0x16, 0xce, 0x0a, 0xac, 0x59, 0x18, 0x68, 0x6d, 0xc6, 0x03, 0xad, 0x3b, 0x67, 0x02, 0xad, 0xa0, - 0x23, 0x45, 0x90, 0xd6, 0x7a, 0x2c, 0xd2, 0xba, 0x7d, 0x16, 0xd2, 0xea, 0x33, 0xb5, 0x1c, 0x6a, - 0xbd, 0x1a, 0x0a, 0xb5, 0xee, 0x8f, 0x03, 0xb5, 0x7c, 0xa1, 0x03, 0x58, 0xeb, 0xf3, 0xe1, 0x58, - 0xeb, 0x1b, 0xe7, 0x38, 0xa2, 0x36, 0x16, 0x6c, 0x7d, 0x6b, 0x00, 0x6c, 0x49, 0xa3, 0xc0, 0x56, - 0x30, 0x32, 0x3c, 0xb4, 0x55, 0x8d, 0xc1, 0x46, 0x37, 0x47, 0x63, 0xa3, 0x60, 0x22, 0x0f, 0xc0, - 0x51, 0x2d, 0x0e, 0x1c, 0xdd, 0x3a, 0x03, 0x1c, 0x05, 0x03, 0x2c, 0x84, 0x8e, 0x1e, 0xf7, 0xa1, - 0xa3, 0xeb, 0x67, 0x46, 0x38, 0x86, 0xe0, 0xd1, 0xe3, 0x3e, 0x78, 0x74, 0x7d, 0x04, 0x3c, 0x0a, - 0x98, 0x39, 0x3e, 0x5a, 0x19, 0xc4, 0x47, 0x37, 0x46, 0xe2, 0x23, 0x5f, 0x42, 0x00, 0x90, 0xd6, - 0x63, 0x01, 0xd2, 0xed, 0xb3, 0x00, 0x52, 0xd0, 0x23, 0xc3, 0x08, 0x69, 0x33, 0x1e, 0x21, 0xdd, - 0x39, 0x13, 0x21, 0xf5, 0xcd, 0x9e, 0x1e, 0x44, 0xaa, 0xc5, 0x41, 0xa4, 0x5b, 0x67, 0x40, 0xa4, - 0xf0, 0xec, 0xe9, 0x63, 0xa4, 0xe6, 0x30, 0x8c, 0x74, 0x6f, 0x0c, 0x8c, 0x14, 0xf8, 0x94, 0x7d, - 0x20, 0xe9, 0xd3, 0x7e, 0x90, 0x24, 0x8d, 0x02, 0x49, 0x41, 0x5f, 0xf6, 0x50, 0xd2, 0x66, 0x3c, - 0x4a, 0xba, 0x73, 0x26, 0x4a, 0x0a, 0x9b, 0x97, 0x10, 0x4c, 0xfa, 0xb4, 0x1f, 0x26, 0x49, 0xa3, - 0x60, 0x52, 0x50, 0x1e, 0x0f, 0x27, 0xd5, 0xe2, 0x70, 0xd2, 0xad, 0x33, 0x70, 0x52, 0x68, 0xd6, - 0x09, 0x80, 0xd2, 0x2f, 0x8f, 0x0f, 0x94, 0x3e, 0x7a, 0xd3, 0xc0, 0x9c, 0xb3, 0x91, 0xd2, 0xa7, - 0xfd, 0x48, 0x49, 0x1a, 0x85, 0x94, 0x02, 0x7d, 0x9c, 0x0f, 0x2a, 0xad, 0x65, 0xb2, 0x6f, 0x8b, - 0xef, 0x48, 0x7f, 0x3e, 0x05, 0x53, 0x35, 0x2f, 0x22, 0x36, 0x74, 0x14, 0x99, 0xf0, 0x26, 0x47, - 0x91, 0xa1, 0x55, 0xd2, 0xb5, 0xa8, 0xc7, 0x74, 0xf6, 0x01, 0x96, 0x83, 0x47, 0x2c, 0x72, 0xd6, - 0x37, 0x38, 0x13, 0x00, 0x7d, 0x00, 0xb3, 0x3d, 0x07, 0xdb, 0x4a, 0xd7, 0x36, 0x2c, 0xdb, 0x70, - 0xd9, 0xbe, 0x27, 0x61, 0x45, 0xfc, 0xf2, 0x64, 0x71, 0x66, 0xdb, 0xc1, 0xf6, 0x16, 0xa7, 0xcb, - 0x33, 0xbd, 0xd0, 0x93, 0x77, 0x39, 0xdd, 0xe4, 0xf8, 0x97, 0xd3, 0xbd, 0x00, 0x91, 0x7e, 0x73, - 0x0e, 0x4f, 0x32, 0xec, 0xd8, 0xaf, 0xf8, 0xf9, 0x90, 0xee, 0x4b, 0xf4, 0x72, 0xd2, 0xe3, 0xbf, - 0xe6, 0xec, 0x28, 0x11, 0x35, 0x80, 0x1e, 0xc8, 0xa3, 0x74, 0xad, 0xb6, 0xa1, 0x1d, 0x53, 0xdf, - 0x21, 0x7a, 0xaa, 0xfa, 0xc8, 0xbb, 0x0d, 0x5e, 0xa9, 0x86, 0xbb, 0x45, 0x39, 0x65, 0x38, 0xf4, - 0x7f, 0xa3, 0x07, 0x70, 0xb1, 0xa3, 0x1e, 0xd1, 0xd0, 0x64, 0xc5, 0x73, 0x06, 0x68, 0xb8, 0x30, - 0xbb, 0xa6, 0x0e, 0x75, 0xd4, 0x23, 0x7a, 0x7d, 0x1e, 0x4b, 0xa2, 0x77, 0xdf, 0x5c, 0x87, 0x19, - 0xbe, 0x1f, 0x82, 0x5d, 0x8d, 0x35, 0x47, 0x73, 0xf2, 0x7b, 0x52, 0xd8, 0xed, 0x58, 0xb7, 0xa0, - 0xa0, 0x1b, 0x8e, 0x6b, 0x98, 0x9a, 0xcb, 0x4f, 0xb2, 0x66, 0x47, 0x36, 0xcf, 0x7a, 0x54, 0x76, - 0x5c, 0x75, 0x13, 0xe6, 0xb5, 0xb6, 0xe1, 0xbb, 0x58, 0x6c, 0xd2, 0x9b, 0x1f, 0xda, 0x97, 0x2b, - 0x34, 0x6f, 0xff, 0x07, 0xe1, 0x39, 0x2d, 0x4a, 0x46, 0x15, 0x98, 0x6b, 0xa9, 0x2e, 0x3e, 0x54, - 0x8f, 0x15, 0x6f, 0x7b, 0x65, 0x9e, 0x1e, 0x05, 0xfb, 0xd6, 0xe9, 0xc9, 0xe2, 0xec, 0x53, 0x96, - 0x34, 0xb0, 0xcb, 0x72, 0xb6, 0x15, 0x4a, 0xd0, 0xd1, 0x1d, 0x98, 0x53, 0x9d, 0x63, 0x53, 0xa3, - 0x0d, 0x88, 0x4d, 0xa7, 0xe7, 0x50, 0x0f, 0x39, 0x2b, 0x17, 0x28, 0xb9, 0xe2, 0x51, 0xd1, 0x63, - 0x28, 0xf1, 0x0b, 0x2b, 0x0e, 0x55, 0x5b, 0x57, 0x68, 0xa3, 0x07, 0xc3, 0x43, 0xa4, 0x3c, 0x97, - 0xd9, 0x05, 0x15, 0x24, 0x03, 0x69, 0xe9, 0xf0, 0xa9, 0xcd, 0xd3, 0x62, 0x76, 0x2d, 0x93, 0x05, - 0x31, 0xbf, 0x96, 0xc9, 0xce, 0x88, 0xb3, 0x6b, 0x99, 0x6c, 0x41, 0x9c, 0x93, 0x7e, 0x5d, 0x80, - 0x99, 0xc8, 0x46, 0xb2, 0xc7, 0x7d, 0x1f, 0x91, 0xaf, 0xc4, 0x3b, 0xfb, 0xc3, 0x42, 0xd8, 0xb3, - 0xbc, 0x69, 0xbd, 0x28, 0xf6, 0xc5, 0xe1, 0x2e, 0x1e, 0x5d, 0x0d, 0xf1, 0xc2, 0x68, 0x3c, 0xb6, - 0x8f, 0x33, 0xdf, 0xff, 0xc1, 0xe2, 0x84, 0xf4, 0x17, 0x19, 0x98, 0x8d, 0x6e, 0x1b, 0xab, 0xf7, - 0x95, 0x2b, 0xce, 0xb8, 0x47, 0x38, 0xca, 0x23, 0xae, 0xda, 0xc9, 0x05, 0xb7, 0x4f, 0xb0, 0x62, - 0x5e, 0x1b, 0xf1, 0xa9, 0x3c, 0x5c, 0xce, 0x80, 0xb1, 0xf4, 0x9f, 0xd2, 0xbe, 0x9d, 0x2a, 0xc3, - 0x24, 0x3d, 0x01, 0x8a, 0x17, 0xad, 0xd8, 0x3f, 0x52, 0x88, 0xe7, 0x42, 0xd2, 0x65, 0x96, 0x8d, - 0xd8, 0xb5, 0xe6, 0x1b, 0x1d, 0xb1, 0x18, 0x98, 0xe4, 0xf3, 0x5f, 0x62, 0xd9, 0x63, 0x47, 0x6c, - 0xfe, 0x7f, 0x0c, 0xb5, 0x21, 0xef, 0x43, 0xbf, 0x00, 0x73, 0x9a, 0xd5, 0x6e, 0xb3, 0x39, 0x8b, - 0x8d, 0xd0, 0xc1, 0x43, 0x77, 0x68, 0x11, 0xf8, 0xbd, 0xa5, 0x65, 0xff, 0xfe, 0xd2, 0xb2, 0xcc, - 0xef, 0x2f, 0x0d, 0xc5, 0x41, 0x17, 0x7c, 0x61, 0x6c, 0x60, 0xf7, 0x85, 0x64, 0x4f, 0xbf, 0x49, - 0x48, 0x36, 0x0b, 0xb2, 0xe7, 0x3d, 0xef, 0x8f, 0x05, 0x1e, 0x10, 0xf3, 0xcc, 0xb2, 0xf6, 0x7b, - 0x7e, 0x10, 0x75, 0x29, 0x7c, 0x60, 0x66, 0x10, 0x2d, 0x4a, 0xb7, 0x04, 0xc5, 0x59, 0xe0, 0xd4, - 0x57, 0xb3, 0xc0, 0xd7, 0x61, 0xa6, 0x6b, 0xe3, 0x5d, 0xec, 0x6a, 0x7b, 0x8a, 0xd9, 0xeb, 0xf0, - 0xfd, 0x50, 0x79, 0x8f, 0xb6, 0xd9, 0xeb, 0xa0, 0x7b, 0x20, 0xfa, 0x59, 0x38, 0x9c, 0xf1, 0x4e, - 0x6b, 0xf3, 0xe8, 0x1c, 0xfc, 0x48, 0xff, 0x47, 0x80, 0x85, 0x48, 0x9d, 0xf8, 0x98, 0x5a, 0x83, - 0xbc, 0xee, 0xcf, 0x79, 0x4e, 0x51, 0x38, 0x67, 0x1c, 0x71, 0x98, 0x19, 0x29, 0x70, 0xc9, 0x7b, - 0x2d, 0xbd, 0xb1, 0x21, 0x10, 0x9b, 0x3a, 0xa7, 0xd8, 0x8b, 0x81, 0x9c, 0xd5, 0xd0, 0x0b, 0xfc, - 0x41, 0x96, 0x1e, 0x6b, 0x90, 0x49, 0xbf, 0x2d, 0x80, 0x48, 0x5f, 0xf0, 0x04, 0x63, 0x3d, 0x11, - 0xeb, 0xe6, 0x05, 0xec, 0xa7, 0xc6, 0xdf, 0xf1, 0x14, 0xb9, 0x65, 0x26, 0x1d, 0xbd, 0x65, 0x46, - 0xfa, 0x81, 0x00, 0x05, 0xbf, 0x84, 0xec, 0x26, 0xc8, 0x11, 0xe7, 0xb2, 0xbe, 0xd9, 0x6d, 0x87, - 0xde, 0xf9, 0x31, 0x63, 0x5d, 0x4e, 0x19, 0x3e, 0x3f, 0x86, 0xdd, 0xd2, 0xf7, 0x77, 0xbd, 0x9e, - 0x43, 0x8a, 0x58, 0x09, 0xce, 0x09, 0x79, 0x83, 0xcd, 0x5f, 0x32, 0xbd, 0x44, 0xd7, 0x6a, 0x1f, - 0xb0, 0x23, 0x7f, 0xc6, 0x32, 0x7b, 0x88, 0x87, 0x81, 0x01, 0x5f, 0xf8, 0xd0, 0x9b, 0x0d, 0x7a, - 0xbd, 0x2e, 0xfb, 0xed, 0x48, 0x4f, 0x42, 0x0a, 0xa4, 0x8d, 0x4f, 0xb4, 0x34, 0x96, 0x29, 0xf6, - 0xb4, 0xc4, 0xfa, 0xca, 0x1f, 0x86, 0x5b, 0xa2, 0x7a, 0x40, 0x30, 0xd8, 0x23, 0x48, 0x1f, 0xa8, - 0xed, 0x51, 0x91, 0x54, 0x91, 0x96, 0x93, 0x49, 0x6e, 0xf4, 0x24, 0x72, 0xbc, 0x4a, 0x6a, 0xf8, - 0xaa, 0xc4, 0xa0, 0x4a, 0x23, 0xc7, 0xb0, 0x7c, 0x18, 0xed, 0xeb, 0x23, 0x5f, 0x1f, 0xee, 0xf4, - 0x1f, 0x67, 0x7e, 0xf4, 0x83, 0x45, 0x41, 0xfa, 0x04, 0x90, 0x8c, 0x1d, 0xec, 0xbe, 0xe8, 0x59, - 0x76, 0x70, 0x54, 0x4d, 0x7f, 0x0c, 0xfd, 0x64, 0x7c, 0x0c, 0xbd, 0x74, 0x11, 0x16, 0x22, 0xdc, - 0xcc, 0x58, 0x48, 0x1f, 0xc2, 0x95, 0xa7, 0x96, 0xe3, 0x18, 0x5d, 0x02, 0x7c, 0xe8, 0xa8, 0x24, - 0x53, 0x8b, 0x6f, 0x1e, 0xb3, 0x5d, 0x8a, 0x35, 0x4d, 0x66, 0x46, 0x72, 0xb2, 0xff, 0x2c, 0xfd, - 0xbe, 0x00, 0x97, 0x07, 0x39, 0x99, 0x96, 0xe3, 0xf6, 0xaa, 0x4e, 0x6b, 0x56, 0x70, 0xb4, 0xe1, - 0xd9, 0xbd, 0xd5, 0xcb, 0x4e, 0x1c, 0x29, 0xfe, 0x4e, 0xa5, 0xa3, 0x52, 0xf3, 0xc1, 0xf7, 0xcd, - 0x17, 0x38, 0x79, 0x83, 0x51, 0x03, 0x4b, 0x92, 0x19, 0xcf, 0x92, 0x34, 0x61, 0x6e, 0xcd, 0x32, - 0x4c, 0xe2, 0xaf, 0x79, 0xf5, 0x5d, 0x86, 0xc2, 0x8e, 0x61, 0xaa, 0xf6, 0xb1, 0xe2, 0x05, 0xf0, - 0x09, 0x67, 0x05, 0xf0, 0xc9, 0xb3, 0x8c, 0x83, 0x3f, 0x4a, 0x3f, 0x16, 0x40, 0x0c, 0xc4, 0x72, - 0x8b, 0xfc, 0x2e, 0x80, 0xd6, 0xee, 0x39, 0x2e, 0xb6, 0xbd, 0x56, 0x9a, 0x61, 0x91, 0xf9, 0x15, - 0x46, 0xad, 0xaf, 0xca, 0x39, 0x9e, 0xa1, 0xae, 0xa3, 0x1b, 0xd1, 0x63, 0x3d, 0x26, 0x57, 0xe0, - 0x74, 0xe0, 0x30, 0x0f, 0xd2, 0xec, 0x8e, 0x6b, 0xd9, 0x3e, 0x76, 0xe1, 0xcd, 0xee, 0x5d, 0x54, - 0x40, 0x77, 0xa3, 0x63, 0xba, 0xf9, 0xa6, 0x40, 0xdc, 0x85, 0x03, 0xec, 0x57, 0x29, 0x73, 0x76, - 0x95, 0x18, 0x87, 0x57, 0xa5, 0x7f, 0x25, 0xc0, 0x5c, 0x85, 0xb5, 0x86, 0xdf, 0xc2, 0x23, 0x2c, - 0xda, 0x2a, 0x64, 0xdd, 0x23, 0x53, 0xe9, 0x60, 0xff, 0x72, 0x9f, 0x73, 0x9c, 0x3b, 0x38, 0xed, - 0xb2, 0x47, 0x7a, 0x5f, 0x24, 0xbf, 0xac, 0x9c, 0x0f, 0x97, 0x2b, 0x65, 0x76, 0x9b, 0x79, 0xd9, - 0xbb, 0xcd, 0xbc, 0xbc, 0xca, 0x33, 0x30, 0xa3, 0xfe, 0xfd, 0xff, 0xba, 0x28, 0xc8, 0x3e, 0x13, - 0x9b, 0xf7, 0xef, 0x37, 0x48, 0xaf, 0x1f, 0x98, 0x99, 0x51, 0x01, 0x20, 0x74, 0x6b, 0x13, 0xbf, - 0x1f, 0x7b, 0x79, 0x55, 0xd9, 0xde, 0xac, 0x3c, 0xdf, 0xd8, 0xa8, 0x37, 0x9b, 0xd5, 0x55, 0x51, - 0x40, 0x22, 0xcc, 0x44, 0xee, 0x7c, 0x4a, 0xb1, 0x1b, 0xb3, 0xef, 0xff, 0x35, 0x80, 0xe0, 0xfa, - 0x38, 0x22, 0x6b, 0xbd, 0xfa, 0x99, 0xf2, 0x72, 0xf9, 0xd9, 0x76, 0xb5, 0x21, 0x4e, 0x20, 0x04, - 0x85, 0x95, 0xe5, 0x66, 0xa5, 0xa6, 0xc8, 0xd5, 0xc6, 0xd6, 0xf3, 0xcd, 0x46, 0xd5, 0xbb, 0x69, - 0xfb, 0xfe, 0x2a, 0xcc, 0x84, 0x0f, 0x6f, 0x42, 0x0b, 0x30, 0x57, 0xa9, 0x55, 0x2b, 0xeb, 0xca, - 0xcb, 0xfa, 0xb2, 0xf2, 0x62, 0xbb, 0xba, 0x5d, 0x15, 0x27, 0x68, 0xd1, 0x28, 0xf1, 0xc9, 0xf6, - 0xb3, 0x67, 0xa2, 0x80, 0xe6, 0x20, 0xcf, 0x9e, 0xe9, 0xfd, 0x50, 0x62, 0xea, 0xfe, 0x06, 0xe4, - 0x43, 0xa7, 0x3c, 0x93, 0xd7, 0x6d, 0x6d, 0x37, 0x6a, 0x4a, 0xb3, 0xbe, 0x51, 0x6d, 0x34, 0x97, - 0x37, 0xb6, 0x98, 0x0c, 0x4a, 0x5b, 0x5e, 0x79, 0x2e, 0x37, 0x45, 0xc1, 0x7f, 0x6e, 0x3e, 0xdf, - 0xae, 0xd4, 0xbc, 0x6a, 0x48, 0x99, 0x6c, 0x5a, 0x4c, 0xdf, 0xff, 0x9b, 0x02, 0x5c, 0x1e, 0x72, - 0x90, 0x11, 0xca, 0xc3, 0xf4, 0xb6, 0x49, 0x8f, 0xbc, 0x15, 0x27, 0xd0, 0x6c, 0xe8, 0x2c, 0x23, - 0x51, 0x40, 0x59, 0x76, 0x9a, 0x8c, 0x98, 0x42, 0x53, 0x90, 0x6a, 0x3c, 0x12, 0xd3, 0xa4, 0xa4, - 0xa1, 0xa3, 0x80, 0xc4, 0x0c, 0xca, 0xf1, 0x43, 0x48, 0xc4, 0x49, 0x34, 0x13, 0x9c, 0x05, 0x22, - 0x4e, 0x11, 0x51, 0xfe, 0x99, 0x1a, 0xe2, 0xf4, 0xfd, 0xeb, 0x10, 0x3a, 0xb7, 0x00, 0x01, 0x4c, - 0x3d, 0x53, 0x5d, 0xec, 0xb8, 0xe2, 0x04, 0x9a, 0x86, 0xf4, 0x72, 0xbb, 0x2d, 0x0a, 0x0f, 0xff, - 0x65, 0x06, 0xb2, 0xde, 0x3d, 0x48, 0xe8, 0x19, 0x4c, 0xb2, 0xc5, 0xc5, 0xc5, 0xe1, 0x68, 0x81, - 0x0e, 0xe8, 0xd2, 0xb5, 0xb3, 0xe0, 0x84, 0x34, 0x81, 0xfe, 0x3a, 0xe4, 0x43, 0x5e, 0x14, 0x1a, - 0xba, 0xa0, 0x13, 0xf1, 0x1c, 0x4b, 0xb7, 0xcf, 0xca, 0xe6, 0xcb, 0x7f, 0x05, 0x39, 0xdf, 0xaa, - 0xa3, 0x1b, 0xa3, 0x6c, 0xbe, 0x27, 0x7b, 0xf4, 0xc4, 0x40, 0xc6, 0x9f, 0x34, 0xf1, 0xbe, 0x80, - 0x6c, 0x40, 0x83, 0x06, 0x18, 0xc5, 0x85, 0x71, 0x0d, 0xb5, 0xf0, 0xa5, 0xfb, 0x63, 0xe5, 0x0e, - 0xde, 0x49, 0x94, 0x15, 0xcc, 0x22, 0xf1, 0xca, 0x1a, 0x98, 0xa3, 0xe2, 0x95, 0x15, 0x33, 0x19, - 0x4d, 0xa0, 0x17, 0x90, 0x21, 0xd6, 0x13, 0xc5, 0xf9, 0x95, 0x7d, 0xd6, 0xba, 0x74, 0x63, 0x64, - 0x1e, 0x4f, 0xe4, 0xca, 0xbd, 0x1f, 0xfd, 0xd9, 0xd5, 0x89, 0x1f, 0x9d, 0x5e, 0x15, 0x7e, 0x7c, - 0x7a, 0x55, 0xf8, 0x93, 0xd3, 0xab, 0xc2, 0x9f, 0x9e, 0x5e, 0x15, 0xbe, 0xf7, 0x93, 0xab, 0x13, - 0x3f, 0xfe, 0xc9, 0xd5, 0x89, 0x3f, 0xf9, 0xc9, 0xd5, 0x89, 0xcf, 0xa7, 0x39, 0xf7, 0xce, 0x14, - 0x35, 0x2d, 0x8f, 0xfe, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x92, 0x15, 0x80, 0x99, 0x8f, 0x82, - 0x00, 0x00, + 0x76, 0x9e, 0x9a, 0xa4, 0x24, 0xf2, 0x50, 0xfc, 0x51, 0x69, 0x7e, 0x38, 0xdc, 0xdd, 0xd1, 0x4c, + 0xcf, 0xff, 0xdc, 0x5d, 0x6a, 0x67, 0xe6, 0x6e, 0x76, 0xbd, 0xb3, 0xde, 0x6b, 0x89, 0xe2, 0x0c, + 0x29, 0x8d, 0x34, 0x9a, 0x26, 0x35, 0x83, 0x5d, 0x5f, 0xa7, 0xdd, 0xea, 0x2e, 0x51, 0x7d, 0x45, + 0x76, 0x73, 0xba, 0x9b, 0xfa, 0x59, 0x20, 0x40, 0x1c, 0x1b, 0xce, 0x7d, 0x0a, 0x6e, 0x00, 0x03, + 0xbe, 0x86, 0x83, 0xf8, 0x3a, 0xd7, 0x88, 0x1f, 0x02, 0x24, 0x01, 0x12, 0xe4, 0x0f, 0x89, 0xfd, + 0x12, 0x20, 0x17, 0x81, 0x13, 0x5f, 0xbf, 0x19, 0x01, 0xa2, 0xd8, 0xba, 0x79, 0x48, 0xe0, 0x20, + 0x08, 0xf2, 0x62, 0x60, 0x81, 0x04, 0x41, 0xfd, 0xf4, 0x1f, 0xd9, 0xa4, 0xa8, 0xd9, 0xbe, 0xf1, + 0x02, 0x7e, 0x91, 0x58, 0xa7, 0xea, 0x9c, 0xae, 0x3a, 0x55, 0x75, 0xea, 0x7c, 0xd5, 0xa7, 0xaa, + 0x61, 0xde, 0x32, 0x15, 0x75, 0xaf, 0xb7, 0xb3, 0xa4, 0xf4, 0xf4, 0x4a, 0xcf, 0x32, 0x1d, 0x13, + 0xcd, 0xab, 0xa6, 0xba, 0x4f, 0xc9, 0x15, 0x9e, 0x59, 0xbe, 0xbf, 0x7f, 0xb0, 0xb4, 0x7f, 0x60, + 0x63, 0xeb, 0x00, 0x5b, 0x4b, 0xaa, 0x69, 0xa8, 0x7d, 0xcb, 0xc2, 0x86, 0x7a, 0xbc, 0xd4, 0x31, + 0xd5, 0x7d, 0xfa, 0x47, 0x37, 0xda, 0x8c, 0x3d, 0x5c, 0xd6, 0xc2, 0x8a, 0x66, 0xf7, 0xbb, 0x5d, + 0xc5, 0x3a, 0x5e, 0xb2, 0xec, 0xde, 0xce, 0x12, 0x4f, 0xf0, 0xb2, 0xc8, 0x7d, 0xba, 0xa6, 0x38, + 0x0a, 0xa7, 0x5d, 0x70, 0x69, 0xd8, 0xb2, 0x4c, 0xcb, 0xe6, 0xd4, 0x4b, 0x2e, 0xb5, 0x8b, 0x1d, + 0x25, 0x50, 0xfa, 0x2d, 0xdb, 0x31, 0x2d, 0xa5, 0x8d, 0x97, 0xb0, 0xd1, 0xd6, 0x0d, 0x4c, 0x0a, + 0x1c, 0xa8, 0x2a, 0xcf, 0x7c, 0x3b, 0x32, 0xf3, 0x11, 0xcf, 0x2d, 0xf5, 0x1d, 0xbd, 0xb3, 0xb4, + 0xd7, 0x51, 0x97, 0x1c, 0xbd, 0x8b, 0x6d, 0x47, 0xe9, 0xf6, 0xdc, 0x26, 0xd0, 0x1c, 0xc7, 0x52, + 0x54, 0xdd, 0x68, 0xbb, 0xff, 0x7b, 0x3b, 0x4b, 0x16, 0x56, 0x4d, 0x4b, 0xc3, 0x9a, 0x6c, 0xf7, + 0x14, 0xc3, 0xad, 0x6e, 0xdb, 0x6c, 0x9b, 0xf4, 0xe7, 0x12, 0xf9, 0xc5, 0xa9, 0x57, 0xdb, 0xa6, + 0xd9, 0xee, 0xe0, 0x25, 0x9a, 0xda, 0xe9, 0xef, 0x2e, 0x69, 0x7d, 0x4b, 0x71, 0x74, 0x93, 0x73, + 0x89, 0xff, 0x4c, 0x80, 0x9c, 0x84, 0x5f, 0xf7, 0xb1, 0xed, 0xd4, 0xb1, 0xa2, 0x61, 0x0b, 0x5d, + 0x81, 0xe4, 0x3e, 0x3e, 0x2e, 0x25, 0xaf, 0x09, 0x77, 0xe7, 0x56, 0x66, 0xbf, 0x3c, 0x59, 0x4c, + 0xae, 0xe3, 0x63, 0x89, 0xd0, 0xd0, 0x35, 0x98, 0xc5, 0x86, 0x26, 0x93, 0xec, 0x54, 0x38, 0x7b, + 0x06, 0x1b, 0xda, 0x3a, 0x3e, 0x46, 0xdf, 0x86, 0xb4, 0x4d, 0xa4, 0x19, 0x2a, 0x2e, 0x4d, 0x5f, + 0x13, 0xee, 0x4e, 0xaf, 0xfc, 0xdc, 0x97, 0x27, 0x8b, 0x9f, 0xb4, 0x75, 0x67, 0xaf, 0xbf, 0x53, + 0x51, 0xcd, 0xee, 0x92, 0xd7, 0xa7, 0xda, 0x8e, 0xff, 0x7b, 0xa9, 0xb7, 0xdf, 0x5e, 0x1a, 0xd4, + 0x51, 0xa5, 0x75, 0x64, 0x34, 0xf1, 0x6b, 0xc9, 0x93, 0xb8, 0x96, 0x4a, 0x0b, 0xc5, 0xc4, 0x5a, + 0x2a, 0x9d, 0x28, 0x26, 0xc5, 0x3f, 0x4c, 0x40, 0x5e, 0xc2, 0x76, 0xcf, 0x34, 0x6c, 0xcc, 0x6b, + 0xfe, 0x3e, 0x24, 0x9d, 0x23, 0x83, 0xd6, 0x3c, 0xfb, 0xf0, 0x6a, 0x65, 0x68, 0xf4, 0x54, 0x5a, + 0x96, 0x62, 0xd8, 0x8a, 0x4a, 0x9a, 0x2f, 0x91, 0xa2, 0xe8, 0x23, 0xc8, 0x5a, 0xd8, 0xee, 0x77, + 0x31, 0x55, 0x24, 0x6d, 0x54, 0xf6, 0xe1, 0xe5, 0x08, 0xce, 0x66, 0x4f, 0x31, 0x24, 0x60, 0x65, + 0xc9, 0x6f, 0xd4, 0x84, 0x1c, 0xe7, 0xb4, 0xb0, 0x62, 0x9b, 0x46, 0x69, 0xf6, 0x9a, 0x70, 0x37, + 0xff, 0xb0, 0x12, 0xc1, 0x1b, 0xae, 0x25, 0x49, 0xf6, 0xbb, 0x58, 0xa2, 0x5c, 0xd2, 0x9c, 0x15, + 0x48, 0xa1, 0x2b, 0x90, 0x36, 0xfa, 0x5d, 0xa2, 0x5f, 0x9b, 0x6a, 0x2f, 0x29, 0xcd, 0x1a, 0xfd, + 0xee, 0x3a, 0x3e, 0xb6, 0xd1, 0x5b, 0x90, 0x21, 0x59, 0x3b, 0xc7, 0x0e, 0xb6, 0x4b, 0x69, 0x9a, + 0x47, 0xca, 0xae, 0x90, 0xb4, 0xf8, 0x29, 0xcc, 0x05, 0xa5, 0x22, 0x04, 0x79, 0xa9, 0xd6, 0xdc, + 0xde, 0xa8, 0xc9, 0xdb, 0x9b, 0xeb, 0x9b, 0xcf, 0x5f, 0x6d, 0x16, 0xa7, 0xd0, 0x05, 0x28, 0x72, + 0xda, 0x7a, 0xed, 0x33, 0xf9, 0x59, 0x63, 0xa3, 0xd1, 0x2a, 0x0a, 0xe5, 0xd4, 0x77, 0x7f, 0x78, + 0x75, 0x6a, 0x2d, 0x95, 0x9e, 0x29, 0xce, 0x8a, 0x3f, 0x14, 0x00, 0x9e, 0x62, 0x87, 0x8f, 0x06, + 0xb4, 0x02, 0x33, 0x7b, 0xb4, 0xc6, 0x25, 0x81, 0xaa, 0xe5, 0x5a, 0x64, 0xd3, 0x02, 0x23, 0x67, + 0x25, 0xfd, 0xa3, 0x93, 0xc5, 0xa9, 0x1f, 0x9f, 0x2c, 0x0a, 0x12, 0xe7, 0x44, 0x2f, 0x20, 0xbb, + 0x8f, 0x8f, 0x65, 0x3e, 0x2f, 0x4b, 0x09, 0xaa, 0xa3, 0xf7, 0x03, 0x82, 0xf6, 0x0f, 0x2a, 0xee, + 0x14, 0xad, 0x04, 0xa6, 0x73, 0x85, 0x70, 0x54, 0x9a, 0x8e, 0x85, 0x8d, 0xb6, 0xb3, 0x27, 0xc1, + 0x3e, 0x3e, 0x7e, 0xc6, 0x64, 0x88, 0xbf, 0x2f, 0x40, 0x96, 0xd6, 0x92, 0x29, 0x15, 0x55, 0x07, + 0xaa, 0x79, 0xfd, 0xcc, 0x1e, 0x88, 0xa8, 0x67, 0x05, 0xa6, 0x0f, 0x94, 0x4e, 0x1f, 0xd3, 0x1a, + 0x66, 0x1f, 0x96, 0x22, 0x64, 0xbc, 0x24, 0xf9, 0x12, 0x2b, 0x86, 0x1e, 0xc3, 0x9c, 0x6e, 0x38, + 0xd8, 0x70, 0x64, 0xc6, 0x96, 0x3c, 0x83, 0x2d, 0xcb, 0x4a, 0xd3, 0x84, 0xf8, 0x4f, 0x05, 0x80, + 0xad, 0x7e, 0xac, 0x7a, 0xfe, 0xe6, 0x84, 0xf5, 0x5f, 0x49, 0x11, 0x56, 0xb7, 0x15, 0x97, 0x60, + 0x46, 0x37, 0x3a, 0xba, 0xc1, 0xea, 0x9f, 0x96, 0x78, 0x0a, 0x5d, 0x80, 0xe9, 0x9d, 0x8e, 0x6e, + 0x68, 0x74, 0x3e, 0xa4, 0x25, 0x96, 0x10, 0x25, 0xc8, 0xd2, 0x5a, 0xc7, 0xa8, 0x77, 0xf1, 0x24, + 0x01, 0x17, 0xab, 0xa6, 0xa1, 0xe9, 0x64, 0x4a, 0x2a, 0x9d, 0xaf, 0x85, 0x56, 0xd6, 0xe0, 0x82, + 0x86, 0x7b, 0x16, 0x56, 0x15, 0x07, 0x6b, 0x32, 0x3e, 0xea, 0x4d, 0xd8, 0xc7, 0xc8, 0xe7, 0xaa, + 0x1d, 0xf5, 0x28, 0x8d, 0xcc, 0x5a, 0x22, 0x80, 0xcd, 0xda, 0x19, 0x62, 0x32, 0xa5, 0x34, 0x3e, + 0xea, 0xd1, 0x59, 0x1b, 0xad, 0x66, 0xf4, 0x4d, 0xb8, 0xac, 0x74, 0x3a, 0xe6, 0xa1, 0xac, 0xef, + 0xca, 0x9a, 0x89, 0x6d, 0xd9, 0x30, 0x1d, 0x19, 0x1f, 0xe9, 0xb6, 0x43, 0x4d, 0x42, 0x5a, 0x5a, + 0xa0, 0xd9, 0x8d, 0xdd, 0x55, 0x13, 0xdb, 0x9b, 0xa6, 0x53, 0x23, 0x59, 0x81, 0xae, 0x9c, 0x0d, + 0x76, 0xa5, 0xf8, 0x0b, 0x70, 0x69, 0x50, 0xbf, 0x71, 0xf6, 0xdf, 0x1f, 0x08, 0x90, 0x6f, 0x18, + 0xba, 0xf3, 0xb5, 0xe8, 0x38, 0x4f, 0x9f, 0xc9, 0xa0, 0x3e, 0xef, 0x43, 0x71, 0x57, 0xd1, 0x3b, + 0xcf, 0x8d, 0x96, 0xd9, 0xdd, 0xb1, 0x1d, 0xd3, 0xc0, 0x36, 0x57, 0xf8, 0x10, 0x5d, 0x7c, 0x09, + 0x05, 0xaf, 0x35, 0x71, 0xaa, 0xc9, 0x81, 0x62, 0xc3, 0x50, 0x2d, 0xdc, 0xc5, 0x46, 0xac, 0x7a, + 0x7a, 0x1b, 0x32, 0xba, 0x2b, 0x97, 0xea, 0x2a, 0x29, 0xf9, 0x04, 0xb1, 0x0f, 0xf3, 0x81, 0xa7, + 0xc6, 0x69, 0x2e, 0xc9, 0x62, 0x84, 0x0f, 0x65, 0xbf, 0x8f, 0xc8, 0x62, 0x84, 0x0f, 0x99, 0x79, + 0x6b, 0x42, 0x6e, 0x15, 0x77, 0xb0, 0x83, 0x63, 0x6c, 0xa9, 0xb8, 0x0d, 0x79, 0x57, 0x68, 0x9c, + 0x1d, 0xf3, 0xeb, 0x02, 0x20, 0x2e, 0x57, 0x31, 0xda, 0x71, 0xd6, 0x18, 0x2d, 0x12, 0xd7, 0xc2, + 0xe9, 0x5b, 0x06, 0x5b, 0xce, 0xd9, 0x98, 0x04, 0x46, 0xa2, 0x2b, 0xba, 0x3f, 0x65, 0x53, 0xc1, + 0x29, 0xcb, 0xdd, 0x9b, 0x43, 0x58, 0x08, 0x55, 0x2c, 0xde, 0xee, 0x4b, 0xd1, 0x3a, 0x25, 0xae, + 0x25, 0x83, 0x3e, 0x1c, 0x25, 0x8a, 0xdf, 0x17, 0x60, 0xbe, 0xda, 0xc1, 0x8a, 0x15, 0xbb, 0x46, + 0xbe, 0x05, 0x69, 0x0d, 0x2b, 0x1a, 0x6d, 0x32, 0x9b, 0xd8, 0xef, 0x04, 0xa4, 0x10, 0x4f, 0xb7, + 0xb2, 0xd7, 0x51, 0x2b, 0x2d, 0xd7, 0x07, 0xe6, 0xb3, 0xdb, 0x63, 0x12, 0x3f, 0x03, 0x14, 0xac, + 0x59, 0x9c, 0x03, 0xe1, 0x77, 0x12, 0x80, 0x24, 0x7c, 0x80, 0x2d, 0x27, 0xf6, 0x66, 0xaf, 0x42, + 0xd6, 0x51, 0xac, 0x36, 0x76, 0x64, 0xe2, 0xdd, 0x9f, 0xa7, 0xe5, 0xc0, 0xf8, 0x08, 0x19, 0xb5, + 0xe0, 0x0e, 0x36, 0x94, 0x9d, 0x0e, 0xa6, 0x52, 0xe4, 0x1d, 0xb3, 0x6f, 0x68, 0xb2, 0xee, 0x60, + 0x4b, 0x71, 0x4c, 0x4b, 0x36, 0x7b, 0x8e, 0xde, 0xd5, 0xbf, 0xa0, 0x8e, 0x3d, 0x1f, 0x6a, 0x37, + 0x58, 0x71, 0xc2, 0xbc, 0x42, 0x0a, 0x37, 0x78, 0xd9, 0xe7, 0x81, 0xa2, 0xa8, 0x02, 0x0b, 0x7a, + 0xdb, 0x30, 0x2d, 0x2c, 0xb7, 0x55, 0xd9, 0xd9, 0xb3, 0xb0, 0xbd, 0x67, 0x76, 0xdc, 0x05, 0x69, + 0x9e, 0x65, 0x3d, 0x55, 0x5b, 0x6e, 0x86, 0xf8, 0x39, 0x2c, 0x84, 0xb4, 0x14, 0x67, 0x17, 0xfc, + 0x2f, 0x01, 0xb2, 0x4d, 0x55, 0x31, 0xe2, 0xd4, 0xfd, 0xa7, 0x90, 0xb5, 0x55, 0xc5, 0x90, 0x77, + 0x4d, 0xab, 0xab, 0x38, 0xb4, 0x5d, 0xf9, 0x90, 0xee, 0x3d, 0xff, 0x5e, 0x55, 0x8c, 0x27, 0xb4, + 0x90, 0x04, 0xb6, 0xf7, 0x7b, 0xd0, 0x7f, 0x9d, 0xfe, 0xea, 0xfe, 0x2b, 0x9b, 0xde, 0x6b, 0xa9, + 0x74, 0xb2, 0x98, 0x12, 0xff, 0x5c, 0x80, 0x39, 0xd6, 0xe4, 0x38, 0xa7, 0xf7, 0x07, 0x90, 0xb2, + 0xcc, 0x43, 0x36, 0xbd, 0xb3, 0x0f, 0xdf, 0x8a, 0x10, 0xb1, 0x8e, 0x8f, 0x83, 0xeb, 0x27, 0x2d, + 0x8e, 0x56, 0x80, 0x7b, 0xa9, 0x32, 0xe5, 0x4e, 0x4e, 0xca, 0x0d, 0x8c, 0x4b, 0x22, 0x32, 0xee, + 0x40, 0x61, 0x47, 0x71, 0xd4, 0x3d, 0xd9, 0xe2, 0x95, 0x24, 0x6b, 0x6d, 0xf2, 0xee, 0x9c, 0x94, + 0xa7, 0x64, 0xb7, 0xea, 0x36, 0x69, 0x39, 0x9b, 0x6f, 0x36, 0xfe, 0x4b, 0xd6, 0xe7, 0xff, 0x57, + 0xe0, 0x73, 0xc8, 0x6d, 0xf9, 0x5f, 0xb6, 0xae, 0xff, 0x8d, 0x04, 0x5c, 0xae, 0xee, 0x61, 0x75, + 0xbf, 0x6a, 0x1a, 0xb6, 0x6e, 0x3b, 0x44, 0x77, 0x71, 0xf6, 0xff, 0x5b, 0x90, 0x39, 0xd4, 0x9d, + 0x3d, 0x59, 0xd3, 0x77, 0x77, 0xa9, 0xb5, 0x4d, 0x4b, 0x69, 0x42, 0x58, 0xd5, 0x77, 0x77, 0xd1, + 0x23, 0x48, 0x75, 0x4d, 0x8d, 0x39, 0xf3, 0xf9, 0x87, 0x8b, 0x11, 0xe2, 0x69, 0xd5, 0xec, 0x7e, + 0x77, 0xc3, 0xd4, 0xb0, 0x44, 0x0b, 0xa3, 0xab, 0x00, 0x2a, 0xa1, 0xf6, 0x4c, 0xdd, 0x70, 0xb8, + 0x71, 0x0c, 0x50, 0x50, 0x1d, 0x32, 0x0e, 0xb6, 0xba, 0xba, 0xa1, 0x38, 0xb8, 0x34, 0x4d, 0x95, + 0x77, 0x33, 0xb2, 0xe2, 0xbd, 0x8e, 0xae, 0x2a, 0xab, 0xd8, 0x56, 0x2d, 0xbd, 0xe7, 0x98, 0x16, + 0xd7, 0xa2, 0xcf, 0x2c, 0xfe, 0xad, 0x14, 0x94, 0x86, 0x75, 0x13, 0xe7, 0x08, 0xd9, 0x82, 0x19, + 0x0b, 0xdb, 0xfd, 0x8e, 0xc3, 0xc7, 0xc8, 0xc3, 0x51, 0x2a, 0x88, 0xa8, 0x01, 0xdd, 0xba, 0xe8, + 0x38, 0xbc, 0xda, 0x5c, 0x4e, 0xf9, 0x5f, 0x0b, 0x30, 0xc3, 0x32, 0xd0, 0x03, 0x48, 0x5b, 0x64, + 0x61, 0x90, 0x75, 0x8d, 0xd6, 0x31, 0xb9, 0x72, 0xe9, 0xf4, 0x64, 0x71, 0x96, 0x2e, 0x16, 0x8d, + 0xd5, 0x2f, 0xfd, 0x9f, 0xd2, 0x2c, 0x2d, 0xd7, 0xd0, 0x48, 0x6f, 0xd9, 0x8e, 0x62, 0x39, 0x74, + 0x53, 0x29, 0xc1, 0x10, 0x12, 0x25, 0xac, 0xe3, 0x63, 0xb4, 0x06, 0x33, 0xb6, 0xa3, 0x38, 0x7d, + 0x9b, 0xf7, 0xd7, 0xb9, 0x2a, 0xdb, 0xa4, 0x9c, 0x12, 0x97, 0x40, 0xdc, 0x2d, 0x0d, 0x3b, 0x8a, + 0xde, 0xa1, 0x1d, 0x98, 0x91, 0x78, 0x4a, 0xfc, 0x4d, 0x01, 0x66, 0x58, 0x51, 0x74, 0x19, 0x16, + 0xa4, 0xe5, 0xcd, 0xa7, 0x35, 0xb9, 0xb1, 0xb9, 0x5a, 0x6b, 0xd5, 0xa4, 0x8d, 0xc6, 0xe6, 0x72, + 0xab, 0x56, 0x9c, 0x42, 0x97, 0x00, 0xb9, 0x19, 0xd5, 0xe7, 0x9b, 0xcd, 0x46, 0xb3, 0x55, 0xdb, + 0x6c, 0x15, 0x05, 0xba, 0xa7, 0x42, 0xe9, 0x01, 0x6a, 0x02, 0xdd, 0x84, 0x6b, 0x83, 0x54, 0xb9, + 0xd9, 0x5a, 0x6e, 0x35, 0xe5, 0x5a, 0xb3, 0xd5, 0xd8, 0x58, 0x6e, 0xd5, 0x56, 0x8b, 0xc9, 0x31, + 0xa5, 0xc8, 0x43, 0x24, 0xa9, 0x56, 0x6d, 0x15, 0x53, 0xa2, 0x03, 0x17, 0x25, 0xac, 0x9a, 0xdd, + 0x5e, 0xdf, 0xc1, 0xa4, 0x96, 0x76, 0x9c, 0x33, 0xe5, 0x32, 0xcc, 0x6a, 0xd6, 0xb1, 0x6c, 0xf5, + 0x0d, 0x3e, 0x4f, 0x66, 0x34, 0xeb, 0x58, 0xea, 0x1b, 0xe2, 0x3f, 0x16, 0xe0, 0xd2, 0xe0, 0x63, + 0xe3, 0x1c, 0x84, 0x2f, 0x20, 0xab, 0x68, 0x1a, 0xd6, 0x64, 0x0d, 0x77, 0x1c, 0x85, 0xbb, 0x44, + 0xf7, 0x03, 0x92, 0xf8, 0x56, 0x60, 0xc5, 0xdb, 0x0a, 0xdc, 0x78, 0x59, 0xad, 0xd2, 0x8a, 0xac, + 0x12, 0x0e, 0xd7, 0xfc, 0x50, 0x21, 0x94, 0x22, 0xfe, 0x8f, 0x14, 0xe4, 0x6a, 0x86, 0xd6, 0x3a, + 0x8a, 0x75, 0x2d, 0xb9, 0x04, 0x33, 0xaa, 0xd9, 0xed, 0xea, 0x8e, 0xab, 0x20, 0x96, 0x42, 0x3f, + 0x13, 0x70, 0x65, 0x93, 0x13, 0x38, 0x74, 0xbe, 0x13, 0x8b, 0x7e, 0x11, 0x2e, 0x13, 0xab, 0x69, + 0x19, 0x4a, 0x47, 0x66, 0xd2, 0x64, 0xc7, 0xd2, 0xdb, 0x6d, 0x6c, 0xf1, 0xed, 0xc7, 0xbb, 0x11, + 0xf5, 0x6c, 0x70, 0x8e, 0x2a, 0x65, 0x68, 0xb1, 0xf2, 0xd2, 0x45, 0x3d, 0x8a, 0x8c, 0x3e, 0x01, + 0x20, 0x4b, 0x11, 0xdd, 0xd2, 0xb4, 0xb9, 0x3d, 0x1a, 0xb5, 0xa7, 0xe9, 0x9a, 0x20, 0xc2, 0x40, + 0xd2, 0x36, 0x7a, 0x01, 0x45, 0xdd, 0x90, 0x77, 0x3b, 0x7a, 0x7b, 0xcf, 0x91, 0x0f, 0x2d, 0xdd, + 0xc1, 0x76, 0x69, 0x9e, 0xca, 0x88, 0xea, 0xea, 0x26, 0xdf, 0x9a, 0xd5, 0x5e, 0x91, 0x92, 0x5c, + 0x5a, 0x5e, 0x37, 0x9e, 0x50, 0x7e, 0x4a, 0xb4, 0xd1, 0x12, 0x81, 0x42, 0xaf, 0xfb, 0xba, 0x85, + 0xe5, 0x07, 0x3d, 0x95, 0xee, 0x83, 0xa4, 0x57, 0xf2, 0xa7, 0x27, 0x8b, 0x20, 0x31, 0xf2, 0x83, + 0xad, 0x2a, 0x81, 0x46, 0xec, 0x77, 0x4f, 0x45, 0xaf, 0xe0, 0x5e, 0x60, 0x0b, 0x86, 0x2c, 0xe6, + 0x5c, 0x53, 0x8a, 0x23, 0xef, 0xe9, 0xed, 0x3d, 0x6c, 0xc9, 0xde, 0x4e, 0x39, 0xdd, 0x0c, 0x4d, + 0x4b, 0x37, 0x7d, 0x86, 0xaa, 0x62, 0x30, 0x85, 0x2c, 0x3b, 0x75, 0x5a, 0xd8, 0xeb, 0x06, 0xd2, + 0x9f, 0x3d, 0x53, 0xb7, 0x4d, 0xa3, 0x94, 0x61, 0xfd, 0xc9, 0x52, 0xe8, 0x1e, 0x14, 0x9d, 0x23, + 0x43, 0xde, 0xc3, 0x8a, 0xe5, 0xec, 0x60, 0xc5, 0x21, 0x0b, 0x3f, 0xd0, 0x12, 0x05, 0xe7, 0xc8, + 0xa8, 0x07, 0xc8, 0x6b, 0xa9, 0xf4, 0x6c, 0x31, 0x2d, 0xfe, 0x67, 0x01, 0xf2, 0xee, 0x70, 0x8b, + 0x73, 0x66, 0xdc, 0x85, 0xa2, 0x69, 0x60, 0xb9, 0xb7, 0xa7, 0xd8, 0x98, 0x37, 0x9a, 0x2f, 0x38, + 0x79, 0xd3, 0xc0, 0x5b, 0x84, 0xcc, 0xda, 0x86, 0xb6, 0x60, 0xde, 0x76, 0x94, 0xb6, 0x6e, 0xb4, + 0x03, 0xba, 0x98, 0x9e, 0x1c, 0x5c, 0x14, 0x39, 0xb7, 0x47, 0x0f, 0x79, 0x29, 0x7f, 0x24, 0xc0, + 0xfc, 0xb2, 0xd6, 0xd5, 0x8d, 0x66, 0xaf, 0xa3, 0xc7, 0xba, 0x67, 0x71, 0x13, 0x32, 0x36, 0x91, + 0xe9, 0x1b, 0x7c, 0x1f, 0x81, 0xa6, 0x69, 0x0e, 0xb1, 0xfc, 0xcf, 0xa0, 0x80, 0x8f, 0x7a, 0x3a, + 0x7b, 0x55, 0xc1, 0x80, 0x53, 0x6a, 0xf2, 0xb6, 0xe5, 0x7d, 0x5e, 0x92, 0xc5, 0xdb, 0xf4, 0x19, + 0xa0, 0x60, 0x93, 0xe2, 0xc4, 0x2e, 0x9f, 0xc1, 0x02, 0x15, 0xbd, 0x6d, 0xd8, 0x31, 0xeb, 0x4b, + 0xfc, 0x79, 0xb8, 0x10, 0x16, 0x1d, 0x67, 0xbd, 0x5f, 0xf1, 0x5e, 0xde, 0xc0, 0x56, 0xac, 0xa0, + 0xd7, 0xd3, 0x35, 0x17, 0x1c, 0x67, 0x9d, 0x7f, 0x45, 0x80, 0x2b, 0x54, 0x36, 0x7d, 0x9b, 0xb3, + 0x8b, 0xad, 0x67, 0x58, 0xb1, 0x63, 0x45, 0xec, 0x37, 0x60, 0x86, 0x21, 0x6f, 0x3a, 0x3e, 0xa7, + 0x57, 0xb2, 0xc4, 0x73, 0x69, 0x3a, 0xa6, 0x45, 0x3c, 0x17, 0x9e, 0x25, 0x2a, 0x50, 0x8e, 0xaa, + 0x45, 0x9c, 0x2d, 0xfd, 0xbb, 0x02, 0xcc, 0x73, 0xa7, 0x91, 0x0c, 0xe5, 0xea, 0x1e, 0xf1, 0x99, + 0x50, 0x0d, 0xb2, 0x2a, 0xfd, 0x25, 0x3b, 0xc7, 0x3d, 0x4c, 0xe5, 0xe7, 0xc7, 0xf9, 0x9b, 0x8c, + 0xad, 0x75, 0xdc, 0xc3, 0xc4, 0x69, 0x75, 0x7f, 0x13, 0x45, 0x05, 0x1a, 0x39, 0xd6, 0x63, 0xa5, + 0xf3, 0x88, 0x96, 0x75, 0x5d, 0x3f, 0xae, 0x83, 0x7f, 0x92, 0xe4, 0x4a, 0x60, 0xcf, 0xe0, 0xc5, + 0x63, 0xf5, 0x51, 0x3e, 0x87, 0x4b, 0xc1, 0xa5, 0x20, 0xd0, 0xf0, 0xc4, 0x39, 0x1a, 0x1e, 0xd8, + 0xd1, 0xf7, 0xa9, 0xe8, 0x33, 0x08, 0xec, 0xd9, 0xcb, 0xac, 0x4d, 0x2e, 0xfa, 0x39, 0x8f, 0x3a, + 0xe6, 0x7d, 0x29, 0x8c, 0x6e, 0xa3, 0x2a, 0xa4, 0xf1, 0x51, 0x4f, 0xd6, 0xb0, 0xad, 0x72, 0xc3, + 0x25, 0x46, 0x09, 0x24, 0x55, 0x19, 0xc2, 0x03, 0xb3, 0xf8, 0xa8, 0x47, 0x88, 0x68, 0x9b, 0x2c, + 0xc5, 0xae, 0xab, 0x40, 0xab, 0x6d, 0x9f, 0x0d, 0x2f, 0xfc, 0x91, 0xc2, 0xc5, 0x15, 0x3c, 0x2f, + 0x81, 0x89, 0x10, 0x7f, 0x20, 0xc0, 0x5b, 0x91, 0xbd, 0x16, 0xe7, 0x42, 0xf6, 0x09, 0xa4, 0x68, + 0xe3, 0x13, 0xe7, 0x6c, 0x3c, 0xe5, 0x12, 0xbf, 0x9b, 0xe0, 0x73, 0x5c, 0xc2, 0x1d, 0x93, 0x28, + 0x36, 0xf6, 0x5d, 0xb9, 0xe7, 0x90, 0x3b, 0x30, 0x1d, 0xe2, 0x48, 0xf0, 0x6e, 0x4f, 0x9c, 0xbb, + 0xdb, 0xe7, 0xa8, 0x00, 0xb7, 0xc7, 0x5f, 0xc2, 0xbc, 0x61, 0x1a, 0x72, 0x58, 0xe8, 0xf9, 0xc7, + 0x52, 0xc1, 0x30, 0x8d, 0x97, 0x01, 0xb9, 0x9e, 0x9d, 0x19, 0xd0, 0x44, 0x9c, 0x76, 0xe6, 0x7b, + 0x02, 0x2c, 0x78, 0x3e, 0x4e, 0xcc, 0x1e, 0xf4, 0x07, 0x90, 0x34, 0xcc, 0xc3, 0xf3, 0xec, 0x7a, + 0x92, 0xf2, 0x64, 0xd5, 0x0b, 0xd7, 0x28, 0xce, 0xf6, 0xfe, 0x9b, 0x04, 0x64, 0x9e, 0x56, 0xe3, + 0x6c, 0xe5, 0x27, 0x7c, 0x47, 0x9d, 0xf5, 0x77, 0xd4, 0x68, 0xf7, 0x9e, 0x57, 0x79, 0x5a, 0x5d, + 0xc7, 0xc7, 0xee, 0x68, 0x27, 0x5c, 0x68, 0x19, 0x32, 0xe1, 0xbd, 0xd7, 0x09, 0x35, 0xe5, 0x73, + 0x95, 0x31, 0x4c, 0x53, 0xb9, 0x6e, 0xf4, 0x86, 0x10, 0x11, 0xbd, 0x41, 0x1e, 0xe3, 0x79, 0x8a, + 0x89, 0xf3, 0x3c, 0x26, 0xe0, 0x22, 0x4e, 0x17, 0x67, 0xc4, 0x17, 0x00, 0xa4, 0x39, 0x71, 0x76, + 0xc9, 0xaf, 0x26, 0x21, 0xbf, 0xd5, 0xb7, 0xf7, 0x62, 0x1e, 0x7d, 0x55, 0x80, 0x5e, 0xdf, 0xa6, + 0x78, 0xe1, 0xc8, 0xe0, 0x6d, 0x3e, 0x23, 0x30, 0xc4, 0x6d, 0x34, 0xe3, 0x6b, 0x1d, 0x19, 0xa8, + 0xce, 0x85, 0x60, 0xd9, 0x8f, 0x2e, 0xb9, 0x31, 0x0e, 0xac, 0xb6, 0x8e, 0x8c, 0x0d, 0xec, 0xa1, + 0x54, 0x26, 0x09, 0x13, 0x49, 0x9f, 0xc0, 0x2c, 0x49, 0xc8, 0x8e, 0x79, 0x9e, 0x6e, 0x9e, 0x21, + 0x3c, 0x2d, 0x13, 0x3d, 0x86, 0x0c, 0xe3, 0x26, 0xab, 0xdf, 0x0c, 0x5d, 0xfd, 0xa2, 0xda, 0xc2, + 0xd5, 0x48, 0xd7, 0xbd, 0x34, 0x65, 0x25, 0x6b, 0xdd, 0x05, 0x98, 0xde, 0x35, 0x2d, 0xd5, 0x7d, + 0x3f, 0xcc, 0x12, 0xac, 0x3f, 0xd7, 0x52, 0xe9, 0x74, 0x31, 0xb3, 0x96, 0x4a, 0x67, 0x8a, 0x20, + 0xfe, 0xa6, 0x00, 0x05, 0xaf, 0x23, 0xe2, 0x5c, 0x10, 0xaa, 0x21, 0x2d, 0x9e, 0xbf, 0x2b, 0x88, + 0x02, 0xc5, 0x7f, 0x47, 0x3d, 0x22, 0xd5, 0x3c, 0xa0, 0x3d, 0x13, 0xe7, 0x48, 0x79, 0xcc, 0x62, + 0x87, 0x12, 0xe7, 0xed, 0x5d, 0x1a, 0x46, 0xf4, 0x00, 0x2e, 0xe8, 0x5d, 0x62, 0xcf, 0x75, 0xa7, + 0x73, 0xcc, 0x61, 0x9b, 0x83, 0xdd, 0x17, 0xd1, 0x0b, 0x7e, 0x5e, 0xd5, 0xcd, 0x12, 0x7f, 0x87, + 0x6e, 0x80, 0xfb, 0x2d, 0x89, 0x53, 0xd5, 0x0d, 0xc8, 0x59, 0x4c, 0x34, 0x71, 0x6b, 0xce, 0xa9, + 0xed, 0x39, 0x8f, 0x95, 0x28, 0xfc, 0xb7, 0x13, 0x50, 0x78, 0xd1, 0xc7, 0xd6, 0xf1, 0xd7, 0x49, + 0xdd, 0xb7, 0xa1, 0x70, 0xa8, 0xe8, 0x8e, 0xbc, 0x6b, 0x5a, 0x72, 0xbf, 0xa7, 0x29, 0x8e, 0x1b, + 0xc0, 0x92, 0x23, 0xe4, 0x27, 0xa6, 0xb5, 0x4d, 0x89, 0x08, 0x03, 0xda, 0x37, 0xcc, 0x43, 0x43, + 0x26, 0x64, 0x0a, 0x94, 0x8f, 0x0c, 0xbe, 0x2b, 0xbd, 0xf2, 0xe1, 0x7f, 0x3a, 0x59, 0x7c, 0x34, + 0x51, 0x58, 0x1a, 0x0d, 0xc1, 0xeb, 0xf7, 0x75, 0xad, 0xb2, 0xbd, 0xdd, 0x58, 0x95, 0x8a, 0x54, + 0xe4, 0x2b, 0x26, 0xb1, 0x75, 0x64, 0xd8, 0xe2, 0xdf, 0x4f, 0x40, 0xd1, 0xd7, 0x51, 0x9c, 0x1d, + 0x59, 0x83, 0xec, 0xeb, 0x3e, 0xb6, 0xf4, 0x37, 0xe8, 0x46, 0xe0, 0x8c, 0xc4, 0xec, 0xdc, 0x87, + 0x79, 0xe7, 0xc8, 0x90, 0x59, 0xd0, 0x20, 0x8b, 0x25, 0x71, 0x63, 0x20, 0x0a, 0x0e, 0xa9, 0x33, + 0xa1, 0xd3, 0x38, 0x12, 0x1b, 0x7d, 0x0e, 0x73, 0x21, 0x6d, 0x25, 0xbf, 0x9a, 0xb6, 0xb2, 0x87, + 0x01, 0x45, 0xfd, 0xbe, 0x00, 0x88, 0x2a, 0xaa, 0xc1, 0x5e, 0x1b, 0x7c, 0x5d, 0xc6, 0xd3, 0x5d, + 0x28, 0xd2, 0x10, 0x4f, 0x59, 0xdf, 0x95, 0xbb, 0xba, 0x6d, 0xeb, 0x46, 0x9b, 0x0f, 0xa8, 0x3c, + 0xa5, 0x37, 0x76, 0x37, 0x18, 0x55, 0xfc, 0x6b, 0xb0, 0x10, 0x6a, 0x40, 0x9c, 0x9d, 0x7d, 0x1d, + 0xe6, 0x76, 0xd9, 0x5b, 0x5d, 0x2a, 0x9c, 0xef, 0x38, 0x66, 0x29, 0x8d, 0x3d, 0x4f, 0xfc, 0xb3, + 0x04, 0x5c, 0x90, 0xb0, 0x6d, 0x76, 0x0e, 0x70, 0xfc, 0x2a, 0xac, 0x03, 0x7f, 0x9d, 0x23, 0xbf, + 0x91, 0x26, 0x33, 0x8c, 0x99, 0x2d, 0x73, 0xe1, 0x6d, 0xfb, 0x9b, 0xe3, 0x47, 0xec, 0xf0, 0x46, + 0x3d, 0xdf, 0xa3, 0x4b, 0x85, 0xf6, 0xe8, 0x4c, 0x28, 0xb0, 0x17, 0xd2, 0x9a, 0x6c, 0xe3, 0xd7, + 0x46, 0xbf, 0xeb, 0x82, 0xa1, 0xca, 0xb8, 0x4a, 0x36, 0x18, 0x4b, 0x13, 0xbf, 0xde, 0xec, 0x77, + 0xa9, 0xef, 0xbc, 0x72, 0x89, 0xd4, 0xf7, 0xf4, 0x64, 0x31, 0x1f, 0xca, 0xb3, 0xa5, 0xbc, 0xee, + 0xa5, 0x89, 0x74, 0xf1, 0xdb, 0x70, 0x71, 0x40, 0xd9, 0x71, 0x7a, 0x3c, 0xff, 0x2a, 0x09, 0x57, + 0xc2, 0xe2, 0xe3, 0x86, 0x38, 0x5f, 0xf7, 0x0e, 0xad, 0x43, 0xae, 0xab, 0x1b, 0x6f, 0xb6, 0x7b, + 0x39, 0xd7, 0xd5, 0x0d, 0x7f, 0x5b, 0x37, 0x62, 0x68, 0xcc, 0xfc, 0x54, 0x87, 0x86, 0x02, 0xe5, + 0xa8, 0xbe, 0x8b, 0x73, 0x7c, 0x7c, 0x57, 0x80, 0xb9, 0xb8, 0xb7, 0xe5, 0xde, 0x2c, 0xb0, 0x4e, + 0x6c, 0x41, 0xee, 0xa7, 0xb0, 0x8f, 0xf7, 0xdb, 0x02, 0xa0, 0x96, 0xd5, 0x37, 0x08, 0xa8, 0x7d, + 0x66, 0xb6, 0xe3, 0x6c, 0xe6, 0x05, 0x98, 0xd6, 0x0d, 0x0d, 0x1f, 0xd1, 0x66, 0xa6, 0x24, 0x96, + 0x08, 0xbd, 0x9d, 0x4c, 0x4e, 0xf4, 0x76, 0x52, 0xfc, 0x1c, 0x16, 0x42, 0x55, 0x8c, 0xb3, 0xfd, + 0xff, 0x3d, 0x01, 0x0b, 0xbc, 0x21, 0xb1, 0xef, 0x60, 0x7e, 0x13, 0xa6, 0x3b, 0x44, 0xe6, 0x98, + 0x7e, 0xa6, 0xcf, 0x74, 0xfb, 0x99, 0x16, 0x46, 0x3f, 0x0b, 0xd0, 0xb3, 0xf0, 0x81, 0xcc, 0x58, + 0x93, 0x13, 0xb1, 0x66, 0x08, 0x07, 0x25, 0xa0, 0xef, 0x0b, 0x50, 0x20, 0x13, 0xba, 0x67, 0x99, + 0x3d, 0xd3, 0x26, 0x3e, 0x8b, 0x3d, 0x19, 0xcc, 0x79, 0x71, 0x7a, 0xb2, 0x98, 0xdb, 0xd0, 0x8d, + 0x2d, 0xce, 0xd8, 0x6a, 0x4e, 0x7c, 0x66, 0xc0, 0x3d, 0x39, 0x51, 0xa9, 0x76, 0x4c, 0x75, 0xdf, + 0x7f, 0xdf, 0x46, 0x2c, 0x8b, 0x27, 0xce, 0x16, 0xff, 0x50, 0x80, 0x0b, 0x3f, 0xb5, 0xed, 0xe2, + 0xbf, 0x08, 0x65, 0x8b, 0x2f, 0xa1, 0x48, 0x7f, 0x34, 0x8c, 0x5d, 0x33, 0xce, 0x8d, 0xfb, 0xff, + 0x23, 0xc0, 0x7c, 0x40, 0x70, 0x9c, 0x0e, 0xce, 0x9b, 0xea, 0x29, 0xc7, 0x22, 0x6c, 0x9c, 0xc9, + 0x54, 0x25, 0xcd, 0xf1, 0xe2, 0x6c, 0x50, 0x56, 0x60, 0x0e, 0x13, 0x2b, 0x46, 0xb7, 0x78, 0x77, + 0xd8, 0xb9, 0x95, 0x81, 0x1d, 0xfd, 0xac, 0x57, 0x60, 0xe5, 0x58, 0xfc, 0x79, 0xe2, 0x61, 0x05, + 0x27, 0x65, 0x9c, 0x53, 0xfe, 0x9f, 0x27, 0xe0, 0x52, 0x95, 0xbd, 0x55, 0x77, 0xc3, 0x4c, 0xe2, + 0x1c, 0x88, 0x25, 0x98, 0x3d, 0xc0, 0x96, 0xad, 0x9b, 0x6c, 0xb5, 0xcf, 0x49, 0x6e, 0x12, 0x95, + 0x21, 0x6d, 0x1b, 0x4a, 0xcf, 0xde, 0x33, 0xdd, 0xd7, 0x89, 0x5e, 0xda, 0x0b, 0x89, 0x99, 0x7e, + 0xf3, 0x90, 0x98, 0x99, 0xf1, 0x21, 0x31, 0xb3, 0x5f, 0x21, 0x24, 0x86, 0xbf, 0xbb, 0xfb, 0xf7, + 0x02, 0x5c, 0x1e, 0xd2, 0x5c, 0x9c, 0x83, 0xf3, 0x3b, 0x90, 0x55, 0xb9, 0x60, 0xb2, 0x3e, 0xb0, + 0x17, 0x93, 0x0d, 0x52, 0xec, 0x0d, 0xa1, 0xcf, 0xe9, 0xc9, 0x22, 0xb8, 0x55, 0x6d, 0xac, 0x72, + 0xe5, 0x90, 0xdf, 0x9a, 0xf8, 0xcb, 0x39, 0x28, 0xd4, 0x8e, 0xd8, 0xa6, 0x7c, 0x93, 0x79, 0x25, + 0xe8, 0x09, 0xa4, 0x7b, 0x96, 0x79, 0xa0, 0xbb, 0xcd, 0xc8, 0x87, 0xe2, 0x21, 0xdc, 0x66, 0x0c, + 0x70, 0x6d, 0x71, 0x0e, 0xc9, 0xe3, 0x45, 0x2d, 0xc8, 0x3c, 0x33, 0x55, 0xa5, 0xf3, 0x44, 0xef, + 0xb8, 0x13, 0xed, 0xfd, 0xb3, 0x05, 0x55, 0x3c, 0x9e, 0x2d, 0xc5, 0xd9, 0x73, 0x3b, 0xc1, 0x23, + 0xa2, 0x06, 0xa4, 0xeb, 0x8e, 0xd3, 0x23, 0x99, 0x7c, 0xfe, 0xdd, 0x99, 0x40, 0x28, 0x61, 0x71, + 0x83, 0x78, 0x5d, 0x76, 0xd4, 0x82, 0xf9, 0xa7, 0xf4, 0x48, 0x5a, 0xb5, 0x63, 0xf6, 0xb5, 0xaa, + 0x69, 0xec, 0xea, 0x6d, 0xbe, 0x4c, 0xdc, 0x9e, 0x40, 0xe6, 0xd3, 0x6a, 0x53, 0x1a, 0x16, 0x80, + 0x96, 0x21, 0xdd, 0x7c, 0xc4, 0x85, 0x31, 0x37, 0xf2, 0xd6, 0x04, 0xc2, 0x9a, 0x8f, 0x24, 0x8f, + 0x0d, 0xad, 0x41, 0x76, 0xf9, 0x8b, 0xbe, 0x85, 0xb9, 0x94, 0x99, 0x91, 0xc1, 0x18, 0x83, 0x52, + 0x28, 0x97, 0x14, 0x64, 0x46, 0x4d, 0xc8, 0xbf, 0x32, 0xad, 0xfd, 0x8e, 0xa9, 0xb8, 0x2d, 0x9c, + 0xa5, 0xe2, 0xbe, 0x31, 0x81, 0x38, 0x97, 0x51, 0x1a, 0x10, 0x81, 0xbe, 0x0d, 0x05, 0xd2, 0x19, + 0x2d, 0x65, 0xa7, 0xe3, 0x56, 0x32, 0x4d, 0xa5, 0xbe, 0x3b, 0x81, 0x54, 0x8f, 0xd3, 0x7d, 0xcf, + 0x30, 0x20, 0xaa, 0x2c, 0x41, 0x2e, 0x34, 0x08, 0x10, 0x82, 0x54, 0x8f, 0xf4, 0xb7, 0x40, 0xc3, + 0xa5, 0xe8, 0x6f, 0xf4, 0x1e, 0xcc, 0x1a, 0xa6, 0x86, 0xdd, 0x19, 0x92, 0x5b, 0xb9, 0x70, 0x7a, + 0xb2, 0x38, 0xb3, 0x69, 0x6a, 0xcc, 0x81, 0xe2, 0xbf, 0xa4, 0x19, 0x52, 0xa8, 0xa1, 0x95, 0xaf, + 0x41, 0x8a, 0xf4, 0x3b, 0x31, 0x4c, 0x3b, 0x8a, 0x8d, 0xb7, 0x2d, 0x9d, 0x4b, 0x73, 0x93, 0xe5, + 0x7f, 0x94, 0x80, 0x44, 0xf3, 0x11, 0x81, 0x08, 0x3b, 0x7d, 0x75, 0x1f, 0x3b, 0x3c, 0x9f, 0xa7, + 0x28, 0x74, 0xb0, 0xf0, 0xae, 0xce, 0x3c, 0xb9, 0x8c, 0xc4, 0x53, 0xe8, 0x1d, 0x00, 0x45, 0x55, + 0xb1, 0x6d, 0xcb, 0xee, 0x51, 0xc5, 0x8c, 0x94, 0x61, 0x94, 0x75, 0x7c, 0x4c, 0xd8, 0x6c, 0xac, + 0x5a, 0xd8, 0x71, 0x63, 0xbd, 0x58, 0x8a, 0xb0, 0x39, 0xb8, 0xdb, 0x93, 0x1d, 0x73, 0x1f, 0x1b, + 0x74, 0x9c, 0x64, 0x88, 0xa9, 0xe9, 0xf6, 0x5a, 0x84, 0x40, 0xac, 0x24, 0x36, 0x34, 0xdf, 0xa4, + 0x65, 0x24, 0x2f, 0x4d, 0x44, 0x5a, 0xb8, 0xad, 0xf3, 0x83, 0x7e, 0x19, 0x89, 0xa7, 0x88, 0x96, + 0x94, 0xbe, 0xb3, 0x47, 0x7b, 0x22, 0x23, 0xd1, 0xdf, 0xe8, 0x36, 0x14, 0x58, 0x78, 0xa8, 0x8c, + 0x0d, 0x55, 0xa6, 0xc6, 0x35, 0x43, 0xb3, 0x73, 0x8c, 0x5c, 0x33, 0x54, 0x62, 0x4a, 0xd1, 0x23, + 0xe0, 0x04, 0x79, 0xbf, 0x6b, 0x13, 0x9d, 0x02, 0x29, 0xb5, 0x52, 0x38, 0x3d, 0x59, 0xcc, 0x36, + 0x69, 0xc6, 0xfa, 0x46, 0x93, 0x2c, 0x50, 0xac, 0xd4, 0x7a, 0xd7, 0x6e, 0x68, 0xe5, 0x5f, 0x13, + 0x20, 0xf9, 0xb4, 0xda, 0x3c, 0xb7, 0xca, 0xdc, 0x8a, 0x26, 0x03, 0x15, 0xbd, 0x03, 0x85, 0x1d, + 0xbd, 0xd3, 0xd1, 0x8d, 0x36, 0x71, 0xda, 0xbe, 0x83, 0x55, 0x57, 0x61, 0x79, 0x4e, 0xde, 0x62, + 0x54, 0x74, 0x0d, 0xb2, 0xaa, 0x85, 0x35, 0x6c, 0x38, 0xba, 0xd2, 0xb1, 0xb9, 0xe6, 0x82, 0xa4, + 0xf2, 0x2f, 0x09, 0x30, 0x4d, 0x67, 0x00, 0x7a, 0x1b, 0x32, 0xaa, 0x69, 0x38, 0x8a, 0x6e, 0x70, + 0x53, 0x96, 0x91, 0x7c, 0xc2, 0xc8, 0xea, 0x5d, 0x87, 0x39, 0x45, 0x55, 0xcd, 0xbe, 0xe1, 0xc8, + 0x86, 0xd2, 0xc5, 0xbc, 0x9a, 0x59, 0x4e, 0xdb, 0x54, 0xba, 0x18, 0x2d, 0x82, 0x9b, 0xf4, 0x4e, + 0xa0, 0x66, 0x24, 0xe0, 0xa4, 0x75, 0x7c, 0x5c, 0xfe, 0xb7, 0x02, 0xa4, 0xdd, 0x39, 0x43, 0xaa, + 0xd1, 0xc6, 0x06, 0x8b, 0x79, 0x77, 0xab, 0xe1, 0x11, 0x06, 0x97, 0xca, 0x8c, 0xbf, 0x54, 0x5e, + 0x80, 0x69, 0x87, 0x4c, 0x0b, 0x5e, 0x03, 0x96, 0xa0, 0xdb, 0xe7, 0x1d, 0xa5, 0xcd, 0x76, 0x0f, + 0x33, 0x12, 0x4b, 0x90, 0xc6, 0xf0, 0x28, 0x63, 0xa6, 0x11, 0x9e, 0x22, 0x35, 0x65, 0xb1, 0xb0, + 0x3b, 0xb8, 0xad, 0x1b, 0x74, 0x2c, 0x25, 0x25, 0xa0, 0xa4, 0x15, 0x42, 0x41, 0x6f, 0x41, 0x86, + 0x15, 0xc0, 0x86, 0x46, 0x07, 0x54, 0x52, 0x4a, 0x53, 0x42, 0xcd, 0xd0, 0xca, 0x18, 0x32, 0xde, + 0xe4, 0x24, 0xdd, 0xd6, 0xb7, 0x3d, 0x45, 0xd2, 0xdf, 0xe8, 0x7d, 0xb8, 0xf0, 0xba, 0xaf, 0x74, + 0xf4, 0x5d, 0xba, 0x31, 0x48, 0x0f, 0x05, 0x50, 0x9d, 0xb1, 0x96, 0x20, 0x2f, 0x8f, 0x4a, 0xa0, + 0xaa, 0x73, 0xe7, 0x72, 0xd2, 0x9f, 0xcb, 0xe2, 0xef, 0x0a, 0x30, 0xcf, 0xa2, 0xb6, 0x58, 0x7c, + 0x6e, 0x7c, 0x7e, 0xc8, 0xc7, 0x90, 0xd1, 0x14, 0x47, 0x61, 0x67, 0x6a, 0x13, 0x63, 0xcf, 0xd4, + 0x7a, 0x67, 0x3c, 0x14, 0x47, 0xa1, 0xe7, 0x6a, 0x11, 0xa4, 0xc8, 0x6f, 0x76, 0xfc, 0x58, 0xa2, + 0xbf, 0xc5, 0xcf, 0x00, 0x05, 0x2b, 0x1a, 0xa7, 0x47, 0x76, 0x0f, 0x2e, 0x12, 0x5d, 0xd7, 0x0c, + 0xd5, 0x3a, 0xee, 0x39, 0xba, 0x69, 0x3c, 0xa7, 0x7f, 0x6d, 0x54, 0x0c, 0xbc, 0x47, 0xa3, 0xaf, + 0xcf, 0xc4, 0xdf, 0x9b, 0x81, 0x5c, 0xed, 0xa8, 0x67, 0x5a, 0xb1, 0xee, 0xba, 0xad, 0xc0, 0x2c, + 0xdf, 0x98, 0x18, 0xf3, 0xaa, 0x7c, 0xc0, 0x98, 0xbb, 0x71, 0x02, 0x9c, 0x11, 0xad, 0x00, 0xb0, + 0x18, 0x5a, 0x1a, 0x27, 0x95, 0x3c, 0xc7, 0x9b, 0x3d, 0xca, 0x46, 0xcf, 0x97, 0x6c, 0x42, 0xb6, + 0x7b, 0xa0, 0xaa, 0xf2, 0xae, 0xde, 0x71, 0x78, 0x28, 0x62, 0x74, 0xd4, 0xfc, 0xc6, 0xcb, 0x6a, + 0xf5, 0x09, 0x2d, 0xc4, 0x42, 0xf8, 0xfc, 0xb4, 0x04, 0x44, 0x02, 0xfb, 0x8d, 0xde, 0x05, 0x7e, + 0xd6, 0x49, 0xb6, 0xdd, 0x93, 0x8b, 0x2b, 0xb9, 0xd3, 0x93, 0xc5, 0x8c, 0x44, 0xa9, 0xcd, 0x66, + 0x4b, 0xca, 0xb0, 0x02, 0x4d, 0xdb, 0x41, 0x37, 0x20, 0x67, 0x76, 0x75, 0x47, 0x76, 0x9d, 0x24, + 0xee, 0x51, 0xce, 0x11, 0xa2, 0xeb, 0x44, 0x9d, 0xe7, 0x08, 0xcc, 0xec, 0xe4, 0x47, 0x60, 0xfe, + 0xa6, 0x00, 0x97, 0xb8, 0x22, 0xe5, 0x1d, 0x1a, 0xf6, 0xaf, 0x74, 0x74, 0xe7, 0x58, 0xde, 0x3f, + 0x28, 0xa5, 0xa9, 0xdf, 0xfa, 0x33, 0x91, 0x1d, 0x12, 0x18, 0x07, 0x15, 0xb7, 0x5b, 0x8e, 0x9f, + 0x71, 0xe6, 0xf5, 0x83, 0x9a, 0xe1, 0x58, 0xc7, 0x2b, 0x97, 0x4f, 0x4f, 0x16, 0x17, 0x86, 0x73, + 0x5f, 0x4a, 0x0b, 0xf6, 0x30, 0x0b, 0xaa, 0x03, 0x60, 0x6f, 0x1c, 0xd2, 0x15, 0x23, 0xda, 0xff, + 0x88, 0x1c, 0xb0, 0x52, 0x80, 0x17, 0xdd, 0x85, 0x22, 0x3f, 0x72, 0xb4, 0xab, 0x77, 0xb0, 0x6c, + 0xeb, 0x5f, 0x60, 0xba, 0xb6, 0x24, 0xa5, 0x3c, 0xa3, 0x13, 0x11, 0x4d, 0xfd, 0x0b, 0x5c, 0xfe, + 0x0e, 0x94, 0x46, 0xd5, 0x3e, 0x38, 0x05, 0x32, 0xec, 0x0d, 0xf2, 0x47, 0xe1, 0xed, 0xa3, 0x09, + 0x86, 0x2a, 0xdf, 0x42, 0xfa, 0x38, 0xf1, 0x91, 0x20, 0xfe, 0x83, 0x04, 0xe4, 0x56, 0xfa, 0x9d, + 0xfd, 0xe7, 0xbd, 0x26, 0xbb, 0x7b, 0x81, 0x98, 0x41, 0x66, 0x28, 0x48, 0x05, 0x05, 0x66, 0x06, + 0xa9, 0x25, 0xd0, 0xbf, 0xc0, 0x64, 0x71, 0x0a, 0x44, 0xe7, 0xf0, 0x63, 0x0d, 0xb4, 0x0d, 0x3e, + 0x99, 0x9e, 0x3c, 0xf8, 0x08, 0x4a, 0x81, 0x82, 0x74, 0xaf, 0x47, 0xc6, 0x86, 0x63, 0xe9, 0x98, + 0xed, 0x57, 0x26, 0xa5, 0x40, 0x08, 0x51, 0x83, 0x64, 0xd7, 0x58, 0x2e, 0x6a, 0xc1, 0x1c, 0x29, + 0x78, 0x2c, 0xd3, 0x25, 0xc4, 0xdd, 0x4f, 0x7e, 0x10, 0xd1, 0xac, 0x50, 0xbd, 0x2b, 0x54, 0x3f, + 0x55, 0xca, 0x43, 0x7f, 0x4a, 0x59, 0xec, 0x53, 0xca, 0x9f, 0x42, 0x71, 0xb0, 0x40, 0x50, 0x97, + 0x29, 0xa6, 0xcb, 0x0b, 0x41, 0x5d, 0x26, 0x03, 0x7a, 0x5a, 0x4b, 0xa5, 0x53, 0xc5, 0x69, 0xf1, + 0x4f, 0x93, 0x90, 0x77, 0x87, 0x59, 0x9c, 0x40, 0x67, 0x05, 0xa6, 0xc9, 0xa0, 0x70, 0x03, 0x5e, + 0x6e, 0x8f, 0x19, 0xdd, 0x3c, 0x90, 0x9e, 0x0c, 0x16, 0x17, 0x93, 0x53, 0xd6, 0x38, 0x0c, 0x4e, + 0xf9, 0x97, 0x12, 0x90, 0xa2, 0xd8, 0xe2, 0x01, 0xa4, 0xe8, 0x42, 0x21, 0x4c, 0xb2, 0x50, 0xd0, + 0xa2, 0xde, 0x72, 0x96, 0x08, 0xb8, 0xa6, 0xc4, 0xe7, 0xdb, 0x53, 0x3e, 0x78, 0xf0, 0x90, 0x1a, + 0x9b, 0x39, 0x89, 0xa7, 0xd0, 0x0a, 0x8d, 0xc4, 0x32, 0x2d, 0x07, 0x6b, 0xdc, 0xa7, 0xbf, 0x76, + 0x56, 0xff, 0xba, 0x8b, 0x92, 0xcb, 0x87, 0xae, 0x40, 0x92, 0x58, 0xb1, 0x59, 0x16, 0x54, 0x71, + 0x7a, 0xb2, 0x98, 0x24, 0xf6, 0x8b, 0xd0, 0xd0, 0x12, 0x64, 0xc3, 0x26, 0x83, 0x78, 0x70, 0xd4, + 0x30, 0x06, 0xa6, 0x3b, 0x74, 0xbc, 0xa9, 0xc5, 0xf0, 0x2c, 0xef, 0xe3, 0xff, 0x99, 0x82, 0x5c, + 0xa3, 0x1b, 0xf7, 0x92, 0xb2, 0x1c, 0xee, 0xe1, 0x28, 0x20, 0x14, 0x7a, 0x68, 0x44, 0x07, 0x87, + 0x56, 0xf0, 0xe4, 0xf9, 0x56, 0xf0, 0x4f, 0xa9, 0x17, 0xcd, 0x86, 0xc6, 0xcc, 0xe4, 0x43, 0x63, + 0x16, 0x1b, 0x1a, 0x5d, 0x89, 0x1a, 0xc4, 0xd3, 0xe6, 0x57, 0x60, 0x24, 0x47, 0x60, 0xa6, 0x70, + 0xfd, 0xa9, 0x9f, 0x23, 0x11, 0x1e, 0xff, 0x68, 0x0a, 0x0d, 0xac, 0x09, 0x5b, 0xd4, 0xd9, 0x37, + 0xb7, 0xa8, 0x65, 0x87, 0x0f, 0xd6, 0x8f, 0x21, 0xa9, 0xe9, 0x6e, 0xe7, 0x4c, 0xbe, 0x54, 0x13, + 0xa6, 0x33, 0x46, 0x6d, 0x2a, 0x38, 0x6a, 0xd9, 0x28, 0x29, 0x37, 0x00, 0xfc, 0xb6, 0xa1, 0x6b, + 0x30, 0x63, 0x76, 0x34, 0xf7, 0x6c, 0x4d, 0x6e, 0x25, 0x73, 0x7a, 0xb2, 0x38, 0xfd, 0xbc, 0xa3, + 0x35, 0x56, 0xa5, 0x69, 0xb3, 0xa3, 0x35, 0x34, 0x7a, 0x7f, 0x08, 0x3e, 0x94, 0xbd, 0xc0, 0xbb, + 0x39, 0x69, 0xd6, 0xc0, 0x87, 0xab, 0xd8, 0x56, 0xf9, 0x80, 0xfb, 0x2d, 0x01, 0xf2, 0xae, 0xee, + 0xe2, 0x35, 0x2a, 0x69, 0xbd, 0xcb, 0x27, 0x59, 0xf2, 0x7c, 0x93, 0xcc, 0xe5, 0xe3, 0xe7, 0x9e, + 0x7f, 0x45, 0xe0, 0xa1, 0xd4, 0x4d, 0x55, 0x71, 0x88, 0x53, 0x11, 0xe3, 0xc4, 0xb8, 0x07, 0x45, + 0x4b, 0x31, 0x34, 0xb3, 0xab, 0x7f, 0x81, 0xd9, 0x66, 0xa2, 0xcd, 0xdf, 0xb2, 0x16, 0x3c, 0x3a, + 0xdd, 0xf5, 0xb3, 0xc5, 0xff, 0x26, 0xf0, 0xb0, 0x6b, 0xaf, 0x1a, 0xf1, 0xc6, 0xc2, 0x64, 0xf9, + 0x9b, 0x08, 0x63, 0xd7, 0x74, 0xa3, 0xc6, 0xde, 0x1e, 0x15, 0x23, 0xd9, 0x30, 0x76, 0x4d, 0xf7, + 0xad, 0xbe, 0xe5, 0x12, 0xec, 0xf2, 0xcf, 0xc1, 0x34, 0xcd, 0x7e, 0x03, 0x03, 0xea, 0x85, 0xfa, + 0x13, 0x8d, 0xff, 0x49, 0x02, 0x6e, 0xd2, 0xa6, 0xbe, 0xc4, 0x96, 0xbe, 0x7b, 0xbc, 0x65, 0x99, + 0x0e, 0x56, 0x1d, 0xac, 0xf9, 0x9b, 0xe9, 0x31, 0x76, 0x81, 0x06, 0x19, 0x1e, 0x86, 0xa0, 0x6b, + 0xfc, 0x0e, 0xa1, 0xa7, 0x5f, 0x6d, 0x93, 0x2d, 0xcd, 0xc2, 0x17, 0x1a, 0xab, 0x52, 0x9a, 0x49, + 0x6e, 0x68, 0x68, 0x19, 0x32, 0x3d, 0xb7, 0x19, 0xe7, 0x8a, 0x74, 0xf3, 0xb8, 0xd0, 0x3a, 0x14, + 0x78, 0x45, 0x95, 0x8e, 0x7e, 0x80, 0x65, 0xc5, 0x39, 0xcf, 0x3a, 0x97, 0x63, 0xbc, 0xcb, 0x84, + 0x75, 0xd9, 0x11, 0xff, 0x76, 0x0a, 0x6e, 0x9d, 0xa1, 0xe2, 0x38, 0x87, 0x57, 0x19, 0xd2, 0x07, + 0xe4, 0x41, 0x3a, 0x6f, 0x7d, 0x5a, 0xf2, 0xd2, 0x68, 0x27, 0xe4, 0x2c, 0xed, 0x2a, 0x7a, 0x87, + 0x38, 0x57, 0x2c, 0xb6, 0x78, 0x74, 0xf4, 0x62, 0x74, 0xac, 0x6e, 0xc0, 0xad, 0x7a, 0x42, 0x05, + 0xd1, 0x62, 0x36, 0xfa, 0xae, 0x00, 0x65, 0xf6, 0x40, 0x16, 0xe0, 0x3a, 0xf0, 0x98, 0x14, 0x7d, + 0xcc, 0x6a, 0xc4, 0x63, 0x26, 0xd2, 0x51, 0x25, 0xf0, 0x2c, 0x5e, 0x91, 0x52, 0xf0, 0x69, 0xc1, + 0xaa, 0x94, 0x7f, 0x5d, 0x80, 0x6c, 0x80, 0x80, 0x6e, 0x0f, 0x9d, 0x50, 0xcc, 0x9e, 0x46, 0x1d, + 0x4b, 0xbc, 0x35, 0x74, 0x2c, 0x71, 0x25, 0xfd, 0xe5, 0xc9, 0x62, 0x4a, 0x62, 0xc7, 0x54, 0xdc, + 0x03, 0x8a, 0xd7, 0xfd, 0x0b, 0xb1, 0x92, 0x03, 0x85, 0xdc, 0x1b, 0xb1, 0xe8, 0xc6, 0x91, 0xe2, + 0xbe, 0xfd, 0xa6, 0x1b, 0x47, 0x24, 0x25, 0xfe, 0x46, 0x02, 0xe6, 0x97, 0x35, 0xad, 0xd9, 0xe4, + 0x16, 0x3e, 0xbe, 0x39, 0xe6, 0x42, 0xe8, 0x84, 0x0f, 0xa1, 0xd1, 0x7b, 0x80, 0x34, 0xdd, 0x66, + 0x17, 0xcb, 0xd8, 0x7b, 0x8a, 0x66, 0x1e, 0xfa, 0x41, 0x2e, 0xf3, 0x6e, 0x4e, 0xd3, 0xcd, 0x40, + 0x4d, 0xa0, 0x58, 0x4e, 0xb6, 0x1d, 0xc5, 0x7b, 0x89, 0x77, 0x6b, 0xa2, 0xf3, 0x79, 0x0c, 0xe4, + 0x79, 0x49, 0x29, 0x43, 0xe4, 0xd0, 0x9f, 0x04, 0x95, 0xe8, 0xa4, 0x53, 0x1c, 0x59, 0xb1, 0xdd, + 0x93, 0x65, 0xec, 0x4a, 0x9b, 0x3c, 0xa3, 0x2f, 0xdb, 0xec, 0xc0, 0x18, 0x3b, 0x3d, 0xe2, 0xab, + 0x26, 0x4e, 0xc0, 0xff, 0xf7, 0x04, 0xc8, 0x4b, 0x78, 0xd7, 0xc2, 0x76, 0xac, 0x5b, 0x1e, 0x4f, + 0x60, 0xce, 0x62, 0x52, 0xe5, 0x5d, 0xcb, 0xec, 0x9e, 0xc7, 0x56, 0x64, 0x39, 0xe3, 0x13, 0xcb, + 0xec, 0x72, 0x93, 0xfc, 0x12, 0x0a, 0x5e, 0x1d, 0xe3, 0x6c, 0xfc, 0xef, 0xd2, 0xb3, 0xe7, 0x4c, + 0x70, 0xdc, 0xd1, 0x26, 0xf1, 0x6a, 0x80, 0xbe, 0x86, 0x0b, 0x56, 0x34, 0x4e, 0x35, 0xfc, 0x57, + 0x01, 0xf2, 0xcd, 0xfe, 0x0e, 0xbb, 0x30, 0x2d, 0x3e, 0x0d, 0xd4, 0x20, 0xd3, 0xc1, 0xbb, 0x8e, + 0xfc, 0x46, 0xe7, 0x1e, 0xd2, 0x84, 0x95, 0x9e, 0xfa, 0x78, 0x0a, 0x60, 0xd1, 0xc3, 0x97, 0x54, + 0x4e, 0xf2, 0x9c, 0x72, 0x32, 0x94, 0x97, 0x90, 0xc9, 0xaa, 0x53, 0xf0, 0x9a, 0x19, 0xe7, 0xfa, + 0xf2, 0x2a, 0x64, 0x1d, 0x92, 0xe7, 0xb1, 0x0e, 0xf3, 0x3c, 0xc0, 0x26, 0xda, 0x42, 0x54, 0x60, + 0x81, 0xba, 0x65, 0xb2, 0xd2, 0xeb, 0x75, 0x74, 0x17, 0xcc, 0x53, 0xfb, 0x93, 0x92, 0xe6, 0x69, + 0xd6, 0x32, 0xcb, 0xa1, 0x30, 0x1e, 0xfd, 0xaa, 0x00, 0x73, 0xbb, 0x16, 0xc6, 0x5f, 0x60, 0x99, + 0x9a, 0xe4, 0xc9, 0x22, 0x88, 0x56, 0x49, 0x1d, 0xbe, 0x72, 0x84, 0x41, 0x96, 0x3d, 0xb8, 0x49, + 0x9e, 0x8b, 0x36, 0xa1, 0xa8, 0x76, 0x58, 0xcc, 0x83, 0x17, 0xcd, 0x74, 0x0e, 0xec, 0x53, 0x60, + 0xcc, 0x7e, 0x40, 0xd3, 0x0b, 0x32, 0x99, 0x14, 0x4d, 0xe6, 0x97, 0x54, 0x72, 0xe8, 0x52, 0x19, + 0x71, 0x09, 0x45, 0xe0, 0x6e, 0xcb, 0x8a, 0x84, 0x15, 0x8d, 0x7b, 0xd8, 0x64, 0x5e, 0x79, 0x09, + 0x3e, 0xaf, 0x5e, 0xc1, 0x3c, 0x1d, 0x37, 0x71, 0x9f, 0x25, 0x17, 0x7f, 0x98, 0x00, 0x14, 0x94, + 0xfc, 0xd3, 0x1b, 0x6f, 0x89, 0xf8, 0xc6, 0xdb, 0xbb, 0x80, 0x58, 0x98, 0xac, 0x2d, 0xf7, 0xb0, + 0x25, 0xdb, 0x58, 0x35, 0xf9, 0xf5, 0x61, 0x82, 0x54, 0xe4, 0x39, 0x5b, 0xd8, 0x6a, 0x52, 0x3a, + 0x5a, 0x06, 0xf0, 0xbd, 0x76, 0xbe, 0x28, 0x4e, 0xe2, 0xb4, 0x67, 0x3c, 0xa7, 0x5d, 0xfc, 0x9e, + 0x00, 0xf9, 0x0d, 0xbd, 0x6d, 0x29, 0xb1, 0xde, 0x8e, 0x85, 0x3e, 0x0e, 0xbf, 0xcd, 0xc8, 0x3e, + 0x2c, 0x47, 0x05, 0x76, 0xb1, 0x12, 0x2e, 0xdc, 0xe6, 0x0c, 0x64, 0xad, 0xf1, 0x6a, 0x14, 0xa7, + 0x91, 0xfd, 0x0f, 0x65, 0x98, 0xe3, 0xf5, 0xde, 0x36, 0x74, 0xd3, 0x40, 0x0f, 0x20, 0xd9, 0xe6, + 0x6f, 0xab, 0xb2, 0x91, 0x3b, 0xcb, 0xfe, 0xdd, 0x93, 0xf5, 0x29, 0x89, 0x94, 0x25, 0x2c, 0xbd, + 0xbe, 0x13, 0xe1, 0xc1, 0xfb, 0x67, 0x1d, 0x82, 0x2c, 0xbd, 0xbe, 0x83, 0x9a, 0x50, 0x50, 0xfd, + 0x0b, 0xef, 0x64, 0xc2, 0x9e, 0x1c, 0x89, 0xfb, 0x23, 0xaf, 0x1e, 0xac, 0x4f, 0x49, 0x79, 0x35, + 0x94, 0x81, 0xaa, 0xc1, 0x7b, 0xd6, 0x52, 0x43, 0x81, 0x94, 0xfe, 0x29, 0xfd, 0xf0, 0x1d, 0x6f, + 0xf5, 0xa9, 0xc0, 0x75, 0x6c, 0xe8, 0x63, 0x98, 0xd1, 0xe8, 0x8d, 0x5e, 0xdc, 0x4a, 0x45, 0x75, + 0x74, 0xe8, 0xe2, 0xb4, 0xfa, 0x94, 0xc4, 0x39, 0xd0, 0x1a, 0xcc, 0xb1, 0x5f, 0xcc, 0x87, 0xe6, + 0xb6, 0xe5, 0xd6, 0x68, 0x09, 0x81, 0xd5, 0xbd, 0x3e, 0x25, 0x65, 0x35, 0x9f, 0x8a, 0x9e, 0x42, + 0x56, 0xed, 0x60, 0xc5, 0xe2, 0xa2, 0x6e, 0x8f, 0x3c, 0xfd, 0x39, 0x74, 0x0b, 0x58, 0x7d, 0x4a, + 0x02, 0xd5, 0x23, 0x92, 0x4a, 0x59, 0xf4, 0x32, 0x28, 0x2e, 0xe9, 0xfd, 0x91, 0x95, 0x1a, 0xbe, + 0x59, 0xab, 0x4e, 0x57, 0x7d, 0x8f, 0x8a, 0xbe, 0x09, 0x29, 0x5b, 0x55, 0xdc, 0x3d, 0x9a, 0xab, + 0x23, 0x6e, 0xeb, 0xf1, 0x99, 0x69, 0x69, 0xf4, 0x98, 0xb9, 0xdf, 0xce, 0x91, 0xbb, 0x5d, 0x1e, + 0xa5, 0xd3, 0xd0, 0xad, 0x10, 0x44, 0xa7, 0x98, 0x12, 0x88, 0x1e, 0x14, 0x82, 0x37, 0x64, 0x7a, + 0xae, 0x9a, 0xee, 0x8f, 0x47, 0xeb, 0x61, 0xe8, 0x1c, 0x7c, 0x9d, 0x5e, 0x3d, 0xe1, 0x12, 0xd1, + 0x06, 0xe4, 0x98, 0xa0, 0x3e, 0x3b, 0xa2, 0x5d, 0x5a, 0x1a, 0x19, 0xcd, 0x10, 0x71, 0x48, 0xbc, + 0x3e, 0x25, 0xcd, 0x29, 0x01, 0xb2, 0x5f, 0xaf, 0x2e, 0xb6, 0xda, 0xb8, 0x94, 0x1d, 0x5f, 0xaf, + 0x60, 0x88, 0xa8, 0x57, 0x2f, 0x4a, 0x44, 0xbf, 0x08, 0x17, 0x98, 0x20, 0x87, 0x47, 0xbe, 0xf1, + 0x00, 0xaa, 0x77, 0x46, 0x06, 0x0d, 0x8c, 0x3c, 0x56, 0x5d, 0x9f, 0x92, 0x90, 0x32, 0x94, 0x89, + 0x54, 0xb8, 0xc8, 0x9e, 0xc0, 0xcf, 0xe5, 0x5a, 0xfc, 0x28, 0x69, 0xe9, 0x06, 0x7d, 0xc4, 0x7b, + 0xa3, 0x1e, 0x11, 0x79, 0x5c, 0xb8, 0x3e, 0x25, 0x2d, 0x28, 0xc3, 0xb9, 0x7e, 0x33, 0x2c, 0x7e, + 0x02, 0x92, 0x0f, 0xb7, 0xf7, 0xc6, 0x37, 0x23, 0xea, 0xe4, 0xa8, 0xd7, 0x8c, 0x50, 0x26, 0xe9, + 0x40, 0xef, 0xe6, 0x07, 0x3a, 0x98, 0xe6, 0x46, 0x76, 0x60, 0xc4, 0x31, 0x49, 0xd2, 0x81, 0x7b, + 0x01, 0x32, 0xaa, 0x40, 0xa2, 0xad, 0x96, 0x72, 0x23, 0xd7, 0x07, 0xef, 0x28, 0x60, 0x7d, 0x4a, + 0x4a, 0xb4, 0x55, 0xf4, 0x29, 0xa4, 0xd9, 0xb9, 0xae, 0x23, 0xa3, 0x94, 0x1f, 0x69, 0x70, 0xc3, + 0xa7, 0xe3, 0xea, 0x53, 0x12, 0x3d, 0x4a, 0xc6, 0x07, 0x32, 0x3f, 0xb3, 0x43, 0x45, 0x54, 0xc6, + 0x1c, 0xe7, 0x1e, 0x38, 0x39, 0x45, 0x06, 0x8c, 0xe5, 0x11, 0xd1, 0x16, 0xe4, 0x2d, 0x16, 0xd5, + 0xec, 0x9e, 0x41, 0x28, 0x8e, 0x8c, 0xf5, 0x89, 0x3a, 0x86, 0x50, 0xa7, 0x1b, 0x1f, 0x01, 0x3a, + 0xe9, 0xbb, 0xb0, 0x44, 0xde, 0x77, 0xf3, 0x23, 0xfb, 0x6e, 0x64, 0x48, 0x3c, 0xe9, 0x3b, 0x6b, + 0x28, 0x13, 0x7d, 0x08, 0xd3, 0x6c, 0x9e, 0x20, 0x2a, 0x32, 0x2a, 0x7c, 0x6d, 0x60, 0x8a, 0xb0, + 0xf2, 0xc4, 0x7a, 0x39, 0x3c, 0xb4, 0x57, 0xee, 0x98, 0xed, 0xd2, 0xc2, 0x48, 0xeb, 0x35, 0x1c, + 0xa4, 0x4c, 0xac, 0x97, 0xe3, 0x53, 0xc9, 0x00, 0xb2, 0x58, 0x0e, 0x9f, 0x62, 0x17, 0x46, 0x0e, + 0xa0, 0x88, 0x88, 0xdf, 0x3a, 0x3d, 0x74, 0xe5, 0x93, 0x3d, 0xc3, 0x6a, 0x63, 0x99, 0x1a, 0xc5, + 0x8b, 0xe3, 0x0d, 0x6b, 0xe8, 0x0a, 0x35, 0xcf, 0xb0, 0x32, 0x2a, 0x7a, 0x09, 0x45, 0x7e, 0x8f, + 0x8f, 0xff, 0x72, 0xf5, 0x12, 0x95, 0x77, 0x2f, 0x72, 0x41, 0x8c, 0x0a, 0x4e, 0xac, 0x13, 0x0f, + 0x35, 0x9c, 0x83, 0x3e, 0x83, 0x79, 0x2a, 0x4f, 0x56, 0xfd, 0xab, 0x97, 0x4a, 0xa5, 0xa1, 0x8b, + 0x7c, 0x46, 0xdf, 0xd2, 0xe4, 0x4a, 0x2e, 0xaa, 0x03, 0x59, 0x64, 0x3e, 0xe8, 0x86, 0xee, 0xd0, + 0xb5, 0xbb, 0x3c, 0x72, 0x3e, 0x84, 0xaf, 0x9d, 0x25, 0xf3, 0x41, 0x67, 0x14, 0x32, 0x8c, 0x07, + 0x2c, 0xde, 0xdb, 0x23, 0x87, 0xf1, 0x08, 0x63, 0x97, 0x73, 0x42, 0x76, 0x6e, 0x15, 0x80, 0xe1, + 0x12, 0xea, 0xf9, 0x5d, 0x1d, 0xe9, 0x00, 0x0c, 0x46, 0xe4, 0x12, 0x07, 0xa0, 0xe3, 0xd2, 0xc8, + 0x3c, 0xa5, 0xbb, 0x1e, 0x32, 0x8d, 0xf4, 0x28, 0x2d, 0x8e, 0x9c, 0xa7, 0x43, 0x51, 0x19, 0x64, + 0x9e, 0x1e, 0x7a, 0x44, 0xe2, 0x49, 0xb0, 0x57, 0x53, 0xa5, 0x6b, 0xa3, 0x57, 0xbd, 0xe0, 0x1b, + 0x6a, 0xba, 0xea, 0x51, 0x02, 0xe1, 0x65, 0x3b, 0xee, 0x25, 0x71, 0x24, 0x6f, 0xe8, 0xed, 0x0a, + 0xe1, 0x65, 0x1c, 0x68, 0x19, 0x32, 0xc4, 0x29, 0x3e, 0xa6, 0x66, 0xe6, 0xfa, 0x48, 0x60, 0x3a, + 0x70, 0x5e, 0xb0, 0x3e, 0x25, 0xa5, 0x5f, 0x73, 0x12, 0x19, 0xda, 0x4c, 0x04, 0x37, 0x30, 0xf7, + 0x47, 0x0e, 0xed, 0xe1, 0x83, 0x62, 0x64, 0x68, 0xbf, 0xf6, 0xa9, 0xfe, 0xba, 0x6b, 0xb3, 0x3d, + 0xfa, 0xd2, 0xcd, 0xf1, 0xeb, 0x6e, 0xf8, 0x8d, 0x82, 0xb7, 0xee, 0x72, 0x32, 0x5b, 0x77, 0x35, + 0xd9, 0xb6, 0x59, 0xf8, 0xcf, 0xad, 0x31, 0xeb, 0xee, 0xc0, 0xae, 0x1d, 0x5b, 0x77, 0xb5, 0x26, + 0xe3, 0x24, 0x2e, 0xa8, 0xe5, 0x5e, 0x9e, 0xc5, 0x31, 0xcb, 0x9d, 0x91, 0x2e, 0x68, 0xe4, 0xed, + 0x5e, 0xc4, 0x05, 0xb5, 0x42, 0x19, 0xe8, 0x67, 0x61, 0x96, 0xef, 0x92, 0x94, 0xee, 0x8e, 0x71, + 0xca, 0x83, 0x1b, 0x5b, 0x64, 0x4e, 0x70, 0x1e, 0x66, 0xa1, 0xd8, 0xee, 0x0c, 0xb3, 0xc0, 0xf7, + 0xc6, 0x58, 0xa8, 0xa1, 0x0d, 0x22, 0x66, 0xa1, 0x7c, 0x32, 0xa9, 0x8d, 0xcd, 0x76, 0x16, 0x4a, + 0xdf, 0x18, 0x59, 0x9b, 0xf0, 0x16, 0x0b, 0xa9, 0x0d, 0xe7, 0xa1, 0x2b, 0x16, 0x75, 0x18, 0x98, + 0x76, 0xde, 0x1d, 0xbd, 0x62, 0x0d, 0x62, 0xd5, 0xba, 0xfb, 0x0e, 0x84, 0x69, 0xe5, 0x6f, 0x08, + 0x70, 0x8d, 0x8d, 0x01, 0xba, 0x03, 0x7c, 0x2c, 0x7b, 0x1b, 0xf8, 0x01, 0x20, 0xfe, 0x80, 0x8a, + 0xff, 0xf0, 0xfc, 0xfb, 0xcd, 0xee, 0x13, 0xdf, 0x51, 0xc6, 0x95, 0x23, 0xca, 0xe8, 0x32, 0x04, + 0x55, 0x7a, 0x38, 0x52, 0x19, 0x61, 0xd4, 0x47, 0x94, 0xc1, 0x79, 0x56, 0x66, 0x79, 0x20, 0x80, + 0x77, 0x0c, 0xbb, 0x50, 0x2c, 0xae, 0xa5, 0xd2, 0x97, 0x8b, 0xa5, 0xb5, 0x54, 0xfa, 0x4a, 0xb1, + 0xbc, 0x96, 0x4a, 0xbf, 0x55, 0x7c, 0x5b, 0xfc, 0x87, 0x65, 0xc8, 0xb9, 0xa0, 0x8b, 0x01, 0xaa, + 0x87, 0x41, 0x40, 0x75, 0x75, 0x14, 0xa0, 0xe2, 0x30, 0x8d, 0x23, 0xaa, 0x87, 0x41, 0x44, 0x75, + 0x75, 0x14, 0xa2, 0xf2, 0x79, 0x08, 0xa4, 0x6a, 0x8d, 0x82, 0x54, 0xf7, 0x26, 0x80, 0x54, 0x9e, + 0xa8, 0x41, 0x4c, 0xb5, 0x3a, 0x8c, 0xa9, 0x6e, 0x8e, 0xc7, 0x54, 0x9e, 0xa8, 0x00, 0xa8, 0x7a, + 0x3c, 0x00, 0xaa, 0xae, 0x8f, 0x01, 0x55, 0x1e, 0xbf, 0x8b, 0xaa, 0xd6, 0x23, 0x51, 0xd5, 0xed, + 0xb3, 0x50, 0x95, 0x27, 0x27, 0x04, 0xab, 0xea, 0x51, 0xb0, 0xea, 0xd6, 0x19, 0xb0, 0xca, 0x13, + 0x15, 0xc4, 0x55, 0xeb, 0x91, 0xb8, 0xea, 0xf6, 0x59, 0xb8, 0xca, 0xaf, 0x56, 0x10, 0x58, 0x7d, + 0x10, 0x02, 0x56, 0x8b, 0x23, 0x81, 0x95, 0xc7, 0xcd, 0x90, 0xd5, 0x27, 0x83, 0xc8, 0xea, 0xfa, + 0x18, 0x64, 0xe5, 0x2b, 0x96, 0x43, 0xab, 0x7a, 0x14, 0xb4, 0xba, 0x75, 0x06, 0xb4, 0xf2, 0x75, + 0x11, 0xc0, 0x56, 0x9b, 0xd1, 0xd8, 0xea, 0xce, 0x99, 0xd8, 0xca, 0x93, 0x16, 0x06, 0x57, 0xf5, + 0x28, 0x70, 0x75, 0xeb, 0x0c, 0x70, 0x35, 0x50, 0x33, 0x86, 0xae, 0x94, 0xb1, 0xe8, 0xea, 0xbd, + 0x09, 0xd1, 0x95, 0x27, 0x3a, 0x0a, 0x5e, 0x69, 0xe3, 0xe1, 0x55, 0x65, 0x52, 0x78, 0xe5, 0x3d, + 0x24, 0x12, 0x5f, 0x29, 0x63, 0xf1, 0xd5, 0x7b, 0x13, 0xe2, 0xab, 0x81, 0x86, 0x84, 0x01, 0xd6, + 0x66, 0x34, 0xc0, 0xba, 0x73, 0x26, 0xc0, 0xf2, 0x7b, 0x31, 0x84, 0xb0, 0x96, 0x02, 0x08, 0xeb, + 0x9d, 0x11, 0x08, 0xcb, 0x63, 0x25, 0x10, 0xeb, 0x5b, 0x43, 0x10, 0x4b, 0x1c, 0x07, 0xb1, 0x3c, + 0x5e, 0x0f, 0x63, 0xd5, 0xa3, 0x30, 0xd6, 0xad, 0x33, 0x30, 0x96, 0x3f, 0x6e, 0x02, 0x20, 0xeb, + 0xc5, 0x08, 0x90, 0x75, 0xf7, 0x6c, 0x90, 0xe5, 0xc9, 0x1b, 0x40, 0x59, 0xca, 0x58, 0x94, 0xf5, + 0xde, 0x84, 0x28, 0xcb, 0xef, 0xc1, 0x08, 0x98, 0xf5, 0x51, 0x18, 0x66, 0x5d, 0x1b, 0x0d, 0xb3, + 0x3c, 0x31, 0x1c, 0x67, 0xad, 0x47, 0xe2, 0xac, 0xdb, 0x67, 0xe1, 0x2c, 0xdf, 0x9a, 0x05, 0x81, + 0xd6, 0x66, 0x34, 0xd0, 0xba, 0x73, 0x26, 0xd0, 0xf2, 0x07, 0x52, 0x08, 0x69, 0xad, 0x47, 0x22, + 0xad, 0xdb, 0x67, 0x21, 0xad, 0x01, 0x53, 0xcb, 0xa1, 0xd6, 0xab, 0x91, 0x50, 0xeb, 0xfe, 0x24, + 0x50, 0xcb, 0x13, 0x3a, 0x84, 0xb5, 0x3e, 0x1f, 0x8d, 0xb5, 0xbe, 0x71, 0x8e, 0x1b, 0x71, 0x23, + 0xc1, 0xd6, 0xb7, 0x86, 0xc0, 0x96, 0x38, 0x0e, 0x6c, 0xf9, 0x33, 0xc3, 0x45, 0x5b, 0xb5, 0x08, + 0x6c, 0x74, 0x73, 0x3c, 0x36, 0xf2, 0x17, 0x72, 0x1f, 0x1c, 0xd5, 0xa3, 0xc0, 0xd1, 0xad, 0x33, + 0xc0, 0x91, 0x3f, 0xc1, 0x02, 0xe8, 0xe8, 0xf1, 0x00, 0x3a, 0xba, 0x7e, 0x66, 0x84, 0x63, 0x00, + 0x1e, 0x3d, 0x1e, 0x80, 0x47, 0xd7, 0xc7, 0xc0, 0x23, 0x9f, 0x99, 0xe3, 0xa3, 0x95, 0x61, 0x7c, + 0x74, 0x63, 0x2c, 0x3e, 0xf2, 0x24, 0xf8, 0x00, 0x69, 0x3d, 0x12, 0x20, 0xdd, 0x3e, 0x0b, 0x20, + 0xf9, 0x23, 0x32, 0x88, 0x90, 0x36, 0xa3, 0x11, 0xd2, 0x9d, 0x33, 0x11, 0xd2, 0xc0, 0xea, 0xe9, + 0x42, 0xa4, 0x7a, 0x14, 0x44, 0xba, 0x75, 0x06, 0x44, 0x0a, 0xae, 0x9e, 0x1e, 0x46, 0x6a, 0x8d, + 0xc2, 0x48, 0xf7, 0x26, 0xc0, 0x48, 0xbe, 0x4f, 0x39, 0x00, 0x92, 0x3e, 0x1d, 0x04, 0x49, 0xe2, + 0x38, 0x90, 0xe4, 0x8f, 0x65, 0x17, 0x25, 0x6d, 0x46, 0xa3, 0xa4, 0x3b, 0x67, 0xa2, 0xa4, 0xa0, + 0x79, 0x09, 0xc0, 0xa4, 0x4f, 0x07, 0x61, 0x92, 0x38, 0x0e, 0x26, 0xf9, 0xf5, 0x71, 0x71, 0x52, + 0x3d, 0x0a, 0x27, 0xdd, 0x3a, 0x03, 0x27, 0x05, 0x56, 0x1d, 0x1f, 0x28, 0xfd, 0xf2, 0xe4, 0x40, + 0xe9, 0xa3, 0x37, 0x0d, 0xcc, 0x39, 0x1b, 0x29, 0x7d, 0x3a, 0x88, 0x94, 0xc4, 0x71, 0x48, 0xc9, + 0xd7, 0xc7, 0xf9, 0xa0, 0xd2, 0x5a, 0x2a, 0xfd, 0x76, 0xf1, 0x1d, 0xf1, 0xcf, 0x66, 0x60, 0xa6, + 0xee, 0x46, 0xc4, 0x06, 0x6e, 0x3e, 0x13, 0xde, 0xe4, 0xe6, 0x33, 0xb4, 0x4a, 0x86, 0x16, 0xf5, + 0x98, 0xce, 0xbe, 0x2f, 0x73, 0xf8, 0x46, 0x47, 0xce, 0xfa, 0x06, 0x57, 0x10, 0xa0, 0x0f, 0x20, + 0xd7, 0xb7, 0xb1, 0x25, 0xf7, 0x2c, 0xdd, 0xb4, 0x74, 0x87, 0x9d, 0x7b, 0x12, 0x56, 0x8a, 0x5f, + 0x9e, 0x2c, 0xce, 0x6d, 0xdb, 0xd8, 0xda, 0xe2, 0x74, 0x69, 0xae, 0x1f, 0x48, 0xb9, 0xdf, 0xc2, + 0x9b, 0x9e, 0xfc, 0x5b, 0x78, 0x2f, 0xa0, 0x48, 0xdf, 0x39, 0x07, 0x17, 0x19, 0x76, 0xcb, 0x58, + 0xf4, 0x7a, 0x48, 0xcf, 0x25, 0xba, 0x25, 0xe9, 0x6d, 0x63, 0x05, 0x2b, 0x4c, 0x44, 0x4d, 0xa0, + 0xf7, 0xff, 0xc8, 0x3d, 0xb3, 0xa3, 0xab, 0xc7, 0xd4, 0x77, 0x08, 0x5f, 0xe2, 0x3e, 0xf6, 0x53, + 0x0a, 0xaf, 0x14, 0xdd, 0xd9, 0xa2, 0x9c, 0x12, 0x1c, 0x7a, 0xbf, 0xd1, 0x03, 0xb8, 0xd8, 0x55, + 0x8e, 0x68, 0x68, 0xb2, 0xec, 0x3a, 0x03, 0x34, 0x5c, 0x98, 0x7d, 0x15, 0x0f, 0x75, 0x95, 0x23, + 0xfa, 0xb5, 0x3e, 0x96, 0x45, 0x3f, 0xb5, 0x73, 0x1d, 0xe6, 0xf8, 0x79, 0x08, 0xf6, 0x25, 0xae, + 0x02, 0x2d, 0xc9, 0x3f, 0xcb, 0xc2, 0x3e, 0xc6, 0x75, 0x0b, 0xf2, 0x9a, 0x6e, 0x3b, 0xba, 0xa1, + 0x3a, 0xfc, 0xe2, 0x6c, 0x76, 0x43, 0x74, 0xce, 0xa5, 0xb2, 0xdb, 0xb1, 0x5b, 0x30, 0xaf, 0x76, + 0x74, 0xcf, 0xc5, 0x62, 0x8b, 0xde, 0xfc, 0xc8, 0xb1, 0x5c, 0xa5, 0x65, 0x07, 0x5f, 0x08, 0x17, + 0xd4, 0x30, 0x19, 0x55, 0xa1, 0xd0, 0x56, 0x1c, 0x7c, 0xa8, 0x1c, 0xcb, 0xee, 0xf1, 0xca, 0x2c, + 0x3d, 0xa7, 0xfe, 0xd6, 0xe9, 0xc9, 0x62, 0xee, 0x29, 0xcb, 0x1a, 0x3a, 0x65, 0x99, 0x6b, 0x07, + 0x32, 0x34, 0x74, 0x07, 0x0a, 0x8a, 0x7d, 0x6c, 0xa8, 0xb4, 0x03, 0xb1, 0x61, 0xf7, 0x6d, 0xea, + 0x21, 0xa7, 0xa5, 0x3c, 0x25, 0x57, 0x5d, 0x2a, 0x7a, 0x0c, 0x65, 0xfe, 0x7d, 0x8c, 0x43, 0xc5, + 0xd2, 0x64, 0xda, 0xe9, 0xfe, 0xf4, 0x28, 0x52, 0x9e, 0xcb, 0xec, 0x7b, 0x18, 0xa4, 0x00, 0xe9, + 0xe9, 0xe0, 0x25, 0xd1, 0xb3, 0xc5, 0xf4, 0x5a, 0x2a, 0x0d, 0xc5, 0xec, 0x5a, 0x2a, 0x3d, 0x57, + 0xcc, 0xad, 0xa5, 0xd2, 0xf9, 0x62, 0x41, 0xfc, 0x35, 0x01, 0xe6, 0x42, 0x07, 0xc9, 0x1e, 0x0f, + 0xbc, 0x44, 0xbe, 0x12, 0xed, 0xec, 0x8f, 0x0a, 0x61, 0x4f, 0xf3, 0xae, 0x75, 0xa3, 0xd8, 0x17, + 0x47, 0xbb, 0x78, 0x74, 0x37, 0xc4, 0x0d, 0xa3, 0x71, 0xd9, 0x3e, 0x4e, 0x7d, 0xff, 0x07, 0x8b, + 0x53, 0xe2, 0x9f, 0xa7, 0x20, 0x17, 0x3e, 0x36, 0xd6, 0x18, 0xa8, 0x57, 0x94, 0x71, 0x0f, 0x71, + 0x54, 0xc6, 0x7c, 0xd9, 0x27, 0xe3, 0x7f, 0xec, 0x82, 0x55, 0xf3, 0xda, 0x98, 0x57, 0xe5, 0xc1, + 0x7a, 0xfa, 0x8c, 0xe5, 0xff, 0x98, 0xf4, 0xec, 0x54, 0x05, 0xa6, 0xe9, 0x85, 0x53, 0xbc, 0x6a, + 0xa5, 0xc1, 0x99, 0x42, 0x3c, 0x17, 0x92, 0x2f, 0xb1, 0x62, 0xc4, 0xae, 0xb5, 0xde, 0xe8, 0x46, + 0x47, 0xdf, 0x24, 0x9f, 0xff, 0x9b, 0x99, 0x7d, 0x76, 0xa3, 0xe7, 0xff, 0xc7, 0x50, 0x1b, 0xf2, + 0x3c, 0xf4, 0x0b, 0x50, 0x50, 0xcd, 0x4e, 0x87, 0xad, 0x59, 0x6c, 0x86, 0x0e, 0xdf, 0xf1, 0x43, + 0xab, 0xc0, 0x3f, 0x93, 0x5a, 0xf1, 0x3e, 0x97, 0x5a, 0x91, 0xf8, 0xe7, 0x52, 0x03, 0x71, 0xd0, + 0x79, 0x4f, 0x18, 0x9b, 0xd8, 0x03, 0x21, 0xd9, 0xb3, 0x6f, 0x12, 0x92, 0xcd, 0x82, 0xec, 0xf9, + 0xc8, 0xfb, 0x23, 0x81, 0x07, 0xc4, 0x3c, 0x33, 0xcd, 0xfd, 0xbe, 0x17, 0x44, 0x5d, 0x0e, 0xde, + 0xcf, 0xe9, 0x47, 0x8b, 0xd2, 0x23, 0x41, 0x51, 0x16, 0x38, 0xf1, 0xd5, 0x2c, 0xf0, 0x75, 0x98, + 0xeb, 0x59, 0x78, 0x17, 0x3b, 0xea, 0x9e, 0x6c, 0xf4, 0xbb, 0xfc, 0x3c, 0x54, 0xd6, 0xa5, 0x6d, + 0xf6, 0xbb, 0xe8, 0x1e, 0x14, 0xbd, 0x22, 0x1c, 0xce, 0xb8, 0x97, 0xc3, 0xb9, 0x74, 0x0e, 0x7e, + 0xc4, 0xff, 0x2d, 0xc0, 0x42, 0xa8, 0x4d, 0x7c, 0x4e, 0xad, 0x41, 0x56, 0xf3, 0xd6, 0x3c, 0xbb, + 0x24, 0x9c, 0x33, 0x8e, 0x38, 0xc8, 0x8c, 0x64, 0xb8, 0xe4, 0x3e, 0x96, 0x7e, 0x20, 0xc2, 0x17, + 0x9b, 0x38, 0xa7, 0xd8, 0x8b, 0xbe, 0x9c, 0xd5, 0xc0, 0x03, 0xbc, 0x49, 0x96, 0x9c, 0x68, 0x92, + 0x89, 0xbf, 0x25, 0x40, 0x91, 0x3e, 0xe0, 0x09, 0xc6, 0x5a, 0x2c, 0xd6, 0xcd, 0x0d, 0xd8, 0x4f, + 0x4c, 0x7e, 0xe2, 0x29, 0xf4, 0x51, 0x9b, 0x64, 0xf8, 0xa3, 0x36, 0xe2, 0x0f, 0x04, 0xc8, 0x7b, + 0x35, 0x64, 0x1f, 0x9e, 0x1c, 0x73, 0x0d, 0xec, 0x9b, 0x7d, 0x5c, 0xd1, 0xbd, 0xae, 0x66, 0xa2, + 0x6f, 0x61, 0x06, 0xaf, 0xab, 0x61, 0x1f, 0x05, 0xfc, 0x3b, 0xee, 0xc8, 0x21, 0x55, 0xac, 0xfa, + 0xf7, 0x84, 0xbc, 0xc1, 0xe1, 0x2f, 0x89, 0x7e, 0xb3, 0xd7, 0xec, 0x1c, 0xb0, 0x1b, 0x86, 0x26, + 0x32, 0x7b, 0x88, 0x87, 0x81, 0x01, 0xdf, 0xf8, 0xd0, 0x5a, 0x4d, 0xfa, 0x35, 0x5f, 0xf6, 0xdb, + 0x16, 0x9f, 0x04, 0x14, 0x48, 0x3b, 0x9f, 0x68, 0x69, 0x22, 0x53, 0xec, 0x6a, 0x89, 0x8d, 0x95, + 0x3f, 0x08, 0xf6, 0x44, 0xed, 0x80, 0x60, 0xb0, 0x47, 0x90, 0x3c, 0x50, 0x3a, 0xe3, 0x22, 0xa9, + 0x42, 0x3d, 0x27, 0x91, 0xd2, 0xe8, 0x49, 0xe8, 0x7a, 0x95, 0xc4, 0xe8, 0x5d, 0x89, 0x61, 0x95, + 0x86, 0xae, 0x61, 0xf9, 0x30, 0x3c, 0xd6, 0xc7, 0x3e, 0x3e, 0x38, 0xe8, 0x3f, 0x4e, 0xfd, 0xe8, + 0x07, 0x8b, 0x82, 0xf8, 0x09, 0x20, 0x09, 0xdb, 0xd8, 0x79, 0xd1, 0x37, 0x2d, 0xff, 0xaa, 0x9a, + 0xc1, 0x18, 0xfa, 0xe9, 0xe8, 0x18, 0x7a, 0xf1, 0x22, 0x2c, 0x84, 0xb8, 0x99, 0xb1, 0x10, 0x3f, + 0x84, 0x2b, 0x4f, 0x4d, 0xdb, 0xd6, 0x7b, 0x04, 0xf8, 0xd0, 0x59, 0x49, 0x96, 0x16, 0xcf, 0x3c, + 0xa6, 0x7b, 0x14, 0x6b, 0x1a, 0xcc, 0x8c, 0x64, 0x24, 0x2f, 0x2d, 0xfe, 0x9e, 0x00, 0x97, 0x87, + 0x39, 0x99, 0x96, 0xa3, 0xce, 0xaa, 0xce, 0xaa, 0xa6, 0x7f, 0x93, 0xe2, 0xd9, 0xa3, 0xd5, 0x2d, + 0x4e, 0x1c, 0x29, 0xfe, 0x4c, 0xb9, 0xab, 0x50, 0xf3, 0xc1, 0xcf, 0xcd, 0xe7, 0x39, 0x79, 0x83, + 0x51, 0x7d, 0x4b, 0x92, 0x9a, 0xcc, 0x92, 0xb4, 0xa0, 0xb0, 0x66, 0xea, 0x06, 0xf1, 0xd7, 0xdc, + 0xf6, 0x2e, 0x43, 0x7e, 0x47, 0x37, 0x14, 0xeb, 0x58, 0x76, 0x03, 0xf8, 0x84, 0xb3, 0x02, 0xf8, + 0xa4, 0x1c, 0xe3, 0xe0, 0x49, 0xf1, 0xc7, 0x02, 0x14, 0x7d, 0xb1, 0xdc, 0x22, 0xbf, 0x0b, 0xa0, + 0x76, 0xfa, 0xb6, 0x83, 0x2d, 0xb7, 0x97, 0xe6, 0x58, 0x64, 0x7e, 0x95, 0x51, 0x1b, 0xab, 0x52, + 0x86, 0x17, 0x68, 0x68, 0xe8, 0x46, 0xf8, 0x5a, 0x8f, 0xe9, 0x15, 0x38, 0x1d, 0xba, 0xcc, 0x83, + 0x74, 0xbb, 0xed, 0x98, 0x96, 0x87, 0x5d, 0x78, 0xb7, 0xbb, 0xb7, 0x28, 0xd1, 0xd3, 0xe8, 0x98, + 0x1e, 0xbe, 0xc9, 0x13, 0x77, 0xe1, 0x00, 0x7b, 0x4d, 0x4a, 0x9d, 0xdd, 0x24, 0xc6, 0xe1, 0x36, + 0xe9, 0x5f, 0x0a, 0x50, 0xa8, 0xb2, 0xde, 0xf0, 0x7a, 0x78, 0x8c, 0x45, 0x5b, 0x85, 0xb4, 0x73, + 0x64, 0xc8, 0x5d, 0xec, 0x7d, 0x4b, 0xe8, 0x1c, 0xd7, 0x1c, 0xce, 0x3a, 0x2c, 0x49, 0x3f, 0x4f, + 0xc9, 0xbf, 0x8d, 0xce, 0xa7, 0xcb, 0x95, 0x0a, 0xfb, 0x78, 0x7a, 0xc5, 0xfd, 0x78, 0x7a, 0x65, + 0x95, 0x17, 0x60, 0x46, 0xfd, 0xfb, 0xff, 0x65, 0x51, 0x90, 0x3c, 0x26, 0xb6, 0xee, 0xdf, 0x6f, + 0x92, 0x51, 0x3f, 0xb4, 0x32, 0xa3, 0x3c, 0x40, 0xe0, 0x23, 0x51, 0xfc, 0x73, 0xdc, 0xcb, 0xab, + 0xf2, 0xf6, 0x66, 0xf5, 0xf9, 0xc6, 0x46, 0xa3, 0xd5, 0xaa, 0xad, 0x16, 0x05, 0x54, 0x84, 0xb9, + 0xd0, 0x27, 0xa6, 0x12, 0xec, 0x03, 0xdd, 0xf7, 0xff, 0x0a, 0x80, 0xff, 0xb5, 0x3a, 0x22, 0x6b, + 0xbd, 0xf6, 0x99, 0xfc, 0x72, 0xf9, 0xd9, 0x76, 0xad, 0x59, 0x9c, 0x42, 0x08, 0xf2, 0x2b, 0xcb, + 0xad, 0x6a, 0x5d, 0x96, 0x6a, 0xcd, 0xad, 0xe7, 0x9b, 0xcd, 0x9a, 0xfb, 0x61, 0xef, 0xfb, 0xab, + 0x30, 0x17, 0xbc, 0xbc, 0x09, 0x2d, 0x40, 0xa1, 0x5a, 0xaf, 0x55, 0xd7, 0xe5, 0x97, 0x8d, 0x65, + 0xf9, 0xc5, 0x76, 0x6d, 0xbb, 0x56, 0x9c, 0xa2, 0x55, 0xa3, 0xc4, 0x27, 0xdb, 0xcf, 0x9e, 0x15, + 0x05, 0x54, 0x80, 0x2c, 0x4b, 0xd3, 0xcf, 0x51, 0x15, 0x13, 0xf7, 0x37, 0x20, 0x1b, 0xb8, 0x54, + 0x9a, 0x3c, 0x6e, 0x6b, 0xbb, 0x59, 0x97, 0x5b, 0x8d, 0x8d, 0x5a, 0xb3, 0xb5, 0xbc, 0xb1, 0xc5, + 0x64, 0x50, 0xda, 0xf2, 0xca, 0x73, 0xa9, 0x55, 0x14, 0xbc, 0x74, 0xeb, 0xf9, 0x76, 0xb5, 0xee, + 0x36, 0x43, 0x4c, 0xa5, 0x93, 0xc5, 0xe4, 0xfd, 0xbf, 0x2e, 0xc0, 0xe5, 0x11, 0x17, 0x19, 0xa1, + 0x2c, 0xcc, 0x6e, 0x1b, 0xf4, 0x86, 0xdd, 0xe2, 0x14, 0xca, 0x05, 0xee, 0x32, 0x2a, 0x0a, 0x28, + 0xcd, 0x6e, 0x93, 0x29, 0x26, 0xd0, 0x0c, 0x24, 0x9a, 0x8f, 0x8a, 0x49, 0x52, 0xd3, 0xc0, 0x55, + 0x40, 0xc5, 0x14, 0xca, 0xf0, 0x4b, 0x48, 0x8a, 0xd3, 0x68, 0xce, 0xbf, 0x0b, 0xa4, 0x38, 0x43, + 0x44, 0x79, 0x77, 0x6a, 0x14, 0x67, 0xef, 0x5f, 0x87, 0xc0, 0xbd, 0x05, 0x08, 0x60, 0xe6, 0x99, + 0xe2, 0x60, 0xdb, 0x29, 0x4e, 0xa1, 0x59, 0x48, 0x2e, 0x77, 0x3a, 0x45, 0xe1, 0xe1, 0xbf, 0x48, + 0x41, 0xda, 0xfd, 0xec, 0x12, 0x7a, 0x06, 0xd3, 0x6c, 0x73, 0x71, 0x71, 0x34, 0x5a, 0xa0, 0x13, + 0xba, 0x7c, 0xed, 0x2c, 0x38, 0x21, 0x4e, 0xa1, 0xbf, 0x0a, 0xd9, 0x80, 0x17, 0x85, 0x46, 0x6e, + 0xe8, 0x84, 0x3c, 0xc7, 0xf2, 0xed, 0xb3, 0x8a, 0x79, 0xf2, 0x5f, 0x41, 0xc6, 0xb3, 0xea, 0xe8, + 0xc6, 0x38, 0x9b, 0xef, 0xca, 0x1e, 0xbf, 0x30, 0x90, 0xf9, 0x27, 0x4e, 0xbd, 0x2f, 0x20, 0x0b, + 0xd0, 0xb0, 0x01, 0x46, 0x51, 0x61, 0x5c, 0x23, 0x2d, 0x7c, 0xf9, 0xfe, 0x44, 0xa5, 0xfd, 0x67, + 0x12, 0x65, 0xf9, 0xab, 0x48, 0xb4, 0xb2, 0x86, 0xd6, 0xa8, 0x68, 0x65, 0x45, 0x2c, 0x46, 0x53, + 0xe8, 0x05, 0xa4, 0x88, 0xf5, 0x44, 0x51, 0x7e, 0xe5, 0x80, 0xb5, 0x2e, 0xdf, 0x18, 0x5b, 0xc6, + 0x15, 0xb9, 0x72, 0xef, 0x47, 0x7f, 0x7a, 0x75, 0xea, 0x47, 0xa7, 0x57, 0x85, 0x1f, 0x9f, 0x5e, + 0x15, 0xfe, 0xf8, 0xf4, 0xaa, 0xf0, 0x27, 0xa7, 0x57, 0x85, 0xef, 0xfd, 0xe4, 0xea, 0xd4, 0x8f, + 0x7f, 0x72, 0x75, 0xea, 0x8f, 0x7f, 0x72, 0x75, 0xea, 0xf3, 0x59, 0xce, 0xbd, 0x33, 0x43, 0x4d, + 0xcb, 0xa3, 0xff, 0x17, 0x00, 0x00, 0xff, 0xff, 0xca, 0x8f, 0x18, 0xcf, 0xfe, 0x82, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -11480,6 +11496,23 @@ func (m *LeaseInfoResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.EvaluatedBy != 0 { + i = encodeVarintApi(dAtA, i, uint64(m.EvaluatedBy)) + i-- + dAtA[i] = 0x20 + } + if m.CurrentLease != nil { + { + size, err := m.CurrentLease.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintApi(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } { size, err := m.Lease.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -16645,12 +16678,12 @@ func (m *ContentionEvent) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - n279, err279 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration):]) - if err279 != nil { - return 0, err279 + n280, err280 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration):]) + if err280 != nil { + return 0, err280 } - i -= n279 - i = encodeVarintApi(dAtA, i, uint64(n279)) + i -= n280 + i = encodeVarintApi(dAtA, i, uint64(n280)) i-- dAtA[i] = 0x1a { @@ -17824,6 +17857,13 @@ func (m *LeaseInfoResponse) Size() (n int) { n += 1 + l + sovApi(uint64(l)) l = m.Lease.Size() n += 1 + l + sovApi(uint64(l)) + if m.CurrentLease != nil { + l = m.CurrentLease.Size() + n += 1 + l + sovApi(uint64(l)) + } + if m.EvaluatedBy != 0 { + n += 1 + sovApi(uint64(m.EvaluatedBy)) + } return n } @@ -28882,6 +28922,61 @@ func (m *LeaseInfoResponse) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CurrentLease", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.CurrentLease == nil { + m.CurrentLease = &Lease{} + } + if err := m.CurrentLease.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field EvaluatedBy", wireType) + } + m.EvaluatedBy = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.EvaluatedBy |= StoreID(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 58e112b7ffd6..11a3291daef2 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1195,7 +1195,9 @@ message TransferLeaseRequest { // LeaseInfoRequest is the argument to the LeaseInfo() method, for getting // information about a range's lease. // It's a point request, so it addresses one single range, and returns the lease -// currently in effect for that range. +// currently in effect for that range. This request is commonly set with +// ReadConsistency=INCONSISTENT in order for the request to be served by the +// node to whom the request was sent. message LeaseInfoRequest{ RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; } @@ -1205,6 +1207,19 @@ message LeaseInfoResponse{ // The last lease known by the replica serving the request. It can also be the // tentative future lease, if a lease transfer is in progress. Lease lease = 2 [(gogoproto.nullable) = false]; + // current_lease is set if `lease` represents a tentative future lease. In + // that case, current_lease represents the lease that's currently in effect. + Lease current_lease = 3; + // evaluated_by returns the store that evaluated this request. This + // corresponds to the leaseholder unless ReadConsistency=INCONSISTENT was + // used. The response reflects the evaluator's view of the lease. When the + // client cares to see a particular node's view, it can use this field to + // check whether the node it intended query (by sending the request to that + // node and using ReadConsistency=INCONSISTENT) indeed served it - it's + // possible that even if ReadConsistency=INCONSISTENT was used, the request is + // still not evaluated by the node it was sent to if that node's replica is a + // learner or the node doesn't have a replica at all. + int32 evaluated_by = 4 [(gogoproto.casttype) = "StoreID"]; } // A RequestLeaseResponse is the response to a RequestLease() or TransferLease() diff --git a/pkg/security/certificate_manager.go b/pkg/security/certificate_manager.go index d0d804d55389..b2752f5259b8 100644 --- a/pkg/security/certificate_manager.go +++ b/pkg/security/certificate_manager.go @@ -304,6 +304,18 @@ func (cl CertsLocator) ClientCAKeyPath() string { return filepath.Join(cl.certsDir, "ca-client"+keyExtension) } +// ClientNodeCertPath returns the expected file path for the certificate used +// by other nodes to verify outgoing RPCs from this node. +func (cl CertsLocator) ClientNodeCertPath() string { + return filepath.Join(cl.certsDir, "client.node"+certExtension) +} + +// ClientNodeKeyPath returns the expected file path for the key used +// to sign outgoing RPCs. +func (cl CertsLocator) ClientNodeKeyPath() string { + return filepath.Join(cl.certsDir, "client.node"+keyExtension) +} + // UICACertPath returns the expected file path for the CA certificate // used to verify Admin UI certificates. func (cl CertsLocator) UICACertPath() string { diff --git a/pkg/server/auto_tls_init.go b/pkg/server/auto_tls_init.go index 4c3a7d4ccad8..55838cb90833 100644 --- a/pkg/server/auto_tls_init.go +++ b/pkg/server/auto_tls_init.go @@ -92,49 +92,6 @@ func (sb *ServiceCertificateBundle) loadCACertAndKey(certPath string, keyPath st return } -// LoadUserAuthCACertAndKey loads host certificate and key from disk or fails with error. -func (sb *ServiceCertificateBundle) loadOrCreateUserAuthCACertAndKey( - ctx context.Context, - caCertPath string, - caKeyPath string, - initLifespan time.Duration, - serviceName string, -) (err error) { - log.Ops.Infof(ctx, "attempting to load CA cert: %s", caCertPath) - // Attempt to load cert into ServiceCertificateBundle. - sb.CACertificate, err = loadCertificateFile(caCertPath) - if err != nil { - if oserror.IsNotExist(err) { - log.Ops.Infof(ctx, "not found; auto-generating") - // Certificate not found, attempt to create both cert and key now. - err = sb.createServiceCA(ctx, caCertPath, caKeyPath, initLifespan, serviceName) - if err != nil { - return err - } - - // Both key and cert should now be populated. - return nil - } - - // Some error unrelated to file existence occurred. - return err - } - - log.Ops.Infof(ctx, "found; loading CA key: %s", caKeyPath) - // Load the key only if it exists. - sb.CAKey, err = loadKeyFile(caKeyPath) - if err != nil { - if !oserror.IsNotExist(err) { - // An error returned but it was not that the file didn't exist; - // this is an error. - return err - } - log.Ops.Infof(ctx, "CA key not found") - } - - return nil -} - // loadOrCreateServiceCertificates will attempt to load the service cert/key // into the service bundle. // * If they do not exist: @@ -373,12 +330,18 @@ func (b *CertificateBundle) InitializeFromConfig(ctx context.Context, c base.Con } // Initialize User auth certificates. - if err := b.UserAuth.loadOrCreateUserAuthCACertAndKey( + if err := b.UserAuth.loadOrCreateServiceCertificates( ctx, + cl.ClientNodeCertPath(), + cl.ClientNodeKeyPath(), cl.ClientCACertPath(), cl.ClientCAKeyPath(), + defaultCertLifetime, defaultCALifetime, + security.NodeUser, serviceNameUserAuth, + nil, + true, /* serviceCertIsAlsoValidAsClient */ ); err != nil { return errors.Wrap(err, "failed to load or create User auth certificate(s)") @@ -494,22 +457,22 @@ func (b *CertificateBundle) InitializeNodeFromBundle(ctx context.Context, c base } // Attempt to write ClientCA to disk. - if err := b.InterNode.writeCAOrFail(cl.ClientCACertPath(), cl.ClientCAKeyPath()); err != nil { + if err := b.UserAuth.writeCAOrFail(cl.ClientCACertPath(), cl.ClientCAKeyPath()); err != nil { return errors.Wrap(err, "failed to write ClientCA to disk") } // Attempt to write SQLServiceCA to disk. - if err := b.InterNode.writeCAOrFail(cl.SQLServiceCACertPath(), cl.SQLServiceCAKeyPath()); err != nil { + if err := b.SQLService.writeCAOrFail(cl.SQLServiceCACertPath(), cl.SQLServiceCAKeyPath()); err != nil { return errors.Wrap(err, "failed to write SQLServiceCA to disk") } // Attempt to write RPCServiceCA to disk. - if err := b.InterNode.writeCAOrFail(cl.RPCServiceCACertPath(), cl.RPCServiceCAKeyPath()); err != nil { + if err := b.RPCService.writeCAOrFail(cl.RPCServiceCACertPath(), cl.RPCServiceCAKeyPath()); err != nil { return errors.Wrap(err, "failed to write RPCServiceCA to disk") } // Attempt to write AdminUIServiceCA to disk. - if err := b.InterNode.writeCAOrFail(cl.UICACertPath(), cl.UICAKeyPath()); err != nil { + if err := b.AdminUIService.writeCAOrFail(cl.UICACertPath(), cl.UICAKeyPath()); err != nil { return errors.Wrap(err, "failed to write AdminUIServiceCA to disk") } @@ -644,7 +607,22 @@ func rotateGeneratedCerts(ctx context.Context, c base.Config) error { } } - // TODO(aaron-crl): Should we rotate UserAuth Certs. + // Rotate UserAuth certificate + if b.UserAuth.CACertificate != nil { + err = b.UserAuth.rotateServiceCert( + ctx, + cl.ClientNodeCertPath(), + cl.ClientNodeKeyPath(), + defaultCertLifetime, + security.NodeUser, + serviceNameUserAuth, + nil, + true, /* serviceCertIsAlsoValidAsClient */ + ) + if err != nil { + return errors.Wrap(err, "failed to rotate InterNode cert") + } + } // Rotate SQLService Certs. if b.SQLService.CACertificate != nil { diff --git a/pkg/server/auto_tls_init_test.go b/pkg/server/auto_tls_init_test.go index 1c96ec30a3be..3b944a92369e 100644 --- a/pkg/server/auto_tls_init_test.go +++ b/pkg/server/auto_tls_init_test.go @@ -77,10 +77,13 @@ func loadAllCertsFromDisk(ctx context.Context, cfg base.Config) (CertificateBund return bundleFromDisk, err } - // TODO(aaron-crl): Figure out how to handle client auth case. - //bundleFromDisk.UserAuth.loadOrCreateServiceCertificates( - // ctx, cl.ClientCertPath(), cl.ClientKeyPath(), "", "", 0, 0, security.NodeUser, "", []string{}, - //) + err = bundleFromDisk.UserAuth.loadOrCreateServiceCertificates( + ctx, cl.ClientNodeCertPath(), cl.ClientNodeKeyPath(), "", "", 0, 0, security.NodeUser, "", []string{}, + true, /* serviceCertIsAlsoValidAsClient */ + ) + if err != nil { + return bundleFromDisk, err + } err = bundleFromDisk.SQLService.loadOrCreateServiceCertificates( ctx, cl.SQLServiceCertPath(), cl.SQLServiceKeyPath(), "", "", 0, 0, security.NodeUser, "", []string{}, false, /* serviceCertIsAlsoValidAsClient */ @@ -181,6 +184,13 @@ func compareBundleServiceCerts( cb1.InterNode.HostKey, cb2.InterNode.HostKey, serviceNameInterNode+" Host key") + cmp( + cb1.UserAuth.HostCertificate, + cb2.UserAuth.HostCertificate, serviceNameUserAuth+" Host cert") + cmp( + cb1.UserAuth.HostKey, + cb2.UserAuth.HostKey, serviceNameUserAuth+" Host key") + cmp( cb1.SQLService.HostCertificate, cb2.SQLService.HostCertificate, serviceNameSQL+" Host cert") @@ -378,18 +388,12 @@ func TestRotationOnPartialIntializedNode(t *testing.T) { t.Fatalf("rotation failed; expected err=nil, got: %q", err) } - // Verify that client and UI service host certs are unchanged. + // Verify that the UI service host certs is unchanged. diskBundle, err := loadAllCertsFromDisk(ctx, cfg) if err != nil { t.Fatalf("failed loading certs from disk, got: %q", err) } cmp := certCompareHelper(t, true) - cmp( - certBundle.UserAuth.HostCertificate, - diskBundle.UserAuth.HostCertificate, "UserAuth host cert") - cmp( - certBundle.UserAuth.HostKey, - diskBundle.UserAuth.HostKey, "UserAuth host key") cmp( certBundle.AdminUIService.HostCertificate, diskBundle.AdminUIService.HostCertificate, "AdminUIService host cert") diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index 9730c22ec305..cdcef72f58d3 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -1238,13 +1238,46 @@ func (ts *TestServer) SplitRange( return ts.SplitRangeWithExpiration(splitKey, hlc.MaxTimestamp) } -// GetRangeLease returns the current lease for the range containing key, and a -// timestamp taken from the node. +// LeaseInfo describes a range's current and potentially future lease. +type LeaseInfo struct { + cur, next roachpb.Lease +} + +// Current returns the range's current lease. +func (l LeaseInfo) Current() roachpb.Lease { + return l.cur +} + +// CurrentOrProspective returns the range's potential next lease, if a lease +// request is in progress, or the current lease otherwise. +func (l LeaseInfo) CurrentOrProspective() roachpb.Lease { + if !l.next.Empty() { + return l.next + } + return l.cur +} + +// LeaseInfoOpt enumerates options for GetRangeLease. +type LeaseInfoOpt int + +const ( + // AllowQueryToBeForwardedToDifferentNode specifies that, if the current node + // doesn't have a voter replica, the lease info can come from a different + // node. + AllowQueryToBeForwardedToDifferentNode LeaseInfoOpt = iota + // QueryLocalNodeOnly specifies that an error should be returned if the node + // is not able to serve the lease query (because it doesn't have a voting + // replica). + QueryLocalNodeOnly +) + +// GetRangeLease returns information on the lease for the range containing key, and a +// timestamp taken from the node. The lease is returned regardless of its status. // -// The lease is returned regardless of its status. +// queryPolicy specifies if its OK to forward the request to a different node. func (ts *TestServer) GetRangeLease( - ctx context.Context, key roachpb.Key, -) (_ roachpb.Lease, now hlc.ClockTimestamp, _ error) { + ctx context.Context, key roachpb.Key, queryPolicy LeaseInfoOpt, +) (_ LeaseInfo, now hlc.ClockTimestamp, _ error) { leaseReq := roachpb.LeaseInfoRequest{ RequestHeader: roachpb.RequestHeader{ Key: key, @@ -1262,10 +1295,25 @@ func (ts *TestServer) GetRangeLease( &leaseReq, ) if pErr != nil { - return roachpb.Lease{}, hlc.ClockTimestamp{}, pErr.GoError() + return LeaseInfo{}, hlc.ClockTimestamp{}, pErr.GoError() + } + // Adapt the LeaseInfoResponse format to LeaseInfo. + resp := leaseResp.(*roachpb.LeaseInfoResponse) + if queryPolicy == QueryLocalNodeOnly && resp.EvaluatedBy != ts.GetFirstStoreID() { + // TODO(andrei): Figure out how to deal with nodes with multiple stores. + // This API API should permit addressing the query to a particular store. + return LeaseInfo{}, hlc.ClockTimestamp{}, errors.Errorf( + "request not evaluated locally; evaluated by s%d instead of local s%d", + resp.EvaluatedBy, ts.GetFirstStoreID()) + } + var l LeaseInfo + if resp.CurrentLease != nil { + l.cur = *resp.CurrentLease + l.next = resp.Lease + } else { + l.cur = resp.Lease } - return leaseResp.(*roachpb.LeaseInfoResponse).Lease, ts.Clock().NowAsClockTimestamp(), nil - + return l, ts.Clock().NowAsClockTimestamp(), nil } // ExecutorConfig is part of the TestServerInterface. diff --git a/pkg/testutils/testcluster/testcluster.go b/pkg/testutils/testcluster/testcluster.go index db812d184c35..e06bcc71bef5 100644 --- a/pkg/testutils/testcluster/testcluster.go +++ b/pkg/testutils/testcluster/testcluster.go @@ -696,7 +696,8 @@ func (tc *TestCluster) WaitForVoters( // startKey is start key of range. // // waitForVoter indicates that the method should wait until the targets are full -// voters in the range. +// voters in the range (and they also know that they're voters - i.e. the +// respective replica has caught up with the config change). // // targets are replication target for change replica. func (tc *TestCluster) waitForNewReplicas( @@ -953,29 +954,53 @@ func (tc *TestCluster) MoveRangeLeaseNonCooperatively( // FindRangeLease is similar to FindRangeLeaseHolder but returns a Lease proto // without verifying if the lease is still active. Instead, it returns a time- // stamp taken off the queried node's clock. +// +// DEPRECATED - use FindRangeLeaseEx instead. func (tc *TestCluster) FindRangeLease( rangeDesc roachpb.RangeDescriptor, hint *roachpb.ReplicationTarget, ) (_ roachpb.Lease, now hlc.ClockTimestamp, _ error) { + l, now, err := tc.FindRangeLeaseEx(context.TODO(), rangeDesc, hint) + if err != nil { + return roachpb.Lease{}, hlc.ClockTimestamp{}, err + } + return l.CurrentOrProspective(), now, err +} + +// FindRangeLeaseEx returns information about a range's lease. As opposed to +// FindRangeLeaseHolder, it doesn't check the validity of the lease; instead it +// returns a timestamp from a node's clock. +// +// If hint is not nil, the respective node will be queried. If that node doesn't +// have a replica able to serve a LeaseInfoRequest, an error will be returned. +// If hint is nil, the first node is queried. In either case, if the returned +// lease is not valid, it's possible that the returned lease information is +// stale - i.e. there might be a newer lease unbeknownst to the queried node. +func (tc *TestCluster) FindRangeLeaseEx( + ctx context.Context, rangeDesc roachpb.RangeDescriptor, hint *roachpb.ReplicationTarget, +) (_ server.LeaseInfo, now hlc.ClockTimestamp, _ error) { + var queryPolicy server.LeaseInfoOpt if hint != nil { var ok bool if _, ok = rangeDesc.GetReplicaDescriptor(hint.StoreID); !ok { - return roachpb.Lease{}, hlc.ClockTimestamp{}, errors.Errorf( + return server.LeaseInfo{}, hlc.ClockTimestamp{}, errors.Errorf( "bad hint: %+v; store doesn't have a replica of the range", hint) } + queryPolicy = server.QueryLocalNodeOnly } else { hint = &roachpb.ReplicationTarget{ NodeID: rangeDesc.Replicas().Descriptors()[0].NodeID, StoreID: rangeDesc.Replicas().Descriptors()[0].StoreID} + queryPolicy = server.AllowQueryToBeForwardedToDifferentNode } // Find the server indicated by the hint and send a LeaseInfoRequest through // it. hintServer, err := tc.findMemberServer(hint.StoreID) if err != nil { - return roachpb.Lease{}, hlc.ClockTimestamp{}, errors.Wrapf(err, "bad hint: %+v; no such node", hint) + return server.LeaseInfo{}, hlc.ClockTimestamp{}, errors.Wrapf(err, "bad hint: %+v; no such node", hint) } - return hintServer.GetRangeLease(context.TODO(), rangeDesc.StartKey.AsRawKey()) + return hintServer.GetRangeLease(ctx, rangeDesc.StartKey.AsRawKey(), queryPolicy) } // FindRangeLeaseHolder is part of TestClusterInterface.