diff --git a/pkg/partitionservice/service.go b/pkg/partitionservice/service.go index da800c4e7adfe..36e80f9e632c1 100644 --- a/pkg/partitionservice/service.go +++ b/pkg/partitionservice/service.go @@ -16,11 +16,12 @@ package partitionservice import ( "context" - "sync" + "fmt" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/pb/partition" "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/sql/parsers/dialect" "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" "github.com/matrixorigin/matrixone/pkg/txn/client" ) @@ -32,13 +33,6 @@ var ( type Service struct { cfg Config store PartitionStorage - - mu struct { - sync.RWMutex - // FIXME: MVCC cache instead. Alter table add/drop partition. Or delete cache if main table has ddl while - // log tail applied. - tables map[uint64]metadataCache - } } func NewService( @@ -49,7 +43,6 @@ func NewService( cfg: cfg, store: store, } - s.mu.tables = make(map[uint64]metadataCache) return s } @@ -89,15 +82,48 @@ func (s *Service) Create( ) } -func (s *Service) Delete( +func (s *Service) Redefine( ctx context.Context, tableID uint64, + stmt *tree.PartitionOption, txnOp client.TxnOperator, ) error { - if s.cfg.Disable { - return nil + metadata, ok, err := s.store.GetMetadata( + ctx, + tableID, + txnOp, + ) + if err != nil { + return err + } + if !ok { + return moerr.NewInternalError(ctx, fmt.Sprintf("table %d is not partitioned", tableID)) + } + + def, err := s.store.GetTableDef( + ctx, + tableID, + txnOp, + ) + if err != nil { + return err } + return s.store.Redefine( + ctx, + def, + stmt, + metadata, + txnOp, + ) +} + +func (s *Service) Rename( + ctx context.Context, + tableID uint64, + oldName, newName string, + txnOp client.TxnOperator, +) error { metadata, ok, err := s.store.GetMetadata( ctx, tableID, @@ -107,31 +133,201 @@ func (s *Service) Delete( return err } if !ok { - return nil + return moerr.NewInternalError(ctx, fmt.Sprintf("table %d is not partitioned", tableID)) } - if txnOp != nil { - txnOp.AppendEventCallback( - client.CommitEvent, - func(te client.TxnEvent) { - s.mu.Lock() - delete(s.mu.tables, tableID) - s.mu.Unlock() + def, err := s.store.GetTableDef( + ctx, + tableID, + txnOp, + ) + if err != nil { + return err + } + + return s.store.Rename( + ctx, + def, + oldName, + newName, + metadata, + txnOp, + ) +} + +func (s *Service) AddPartitions( + ctx context.Context, + tableID uint64, + partitions []*tree.Partition, + txnOp client.TxnOperator, +) error { + metadata, ok, err := s.store.GetMetadata( + ctx, + tableID, + txnOp, + ) + if err != nil { + return err + } + if !ok { + return moerr.NewInternalError(ctx, fmt.Sprintf("table %d is not partitioned", tableID)) + } + + def, err := s.store.GetTableDef( + ctx, + tableID, + txnOp, + ) + if err != nil { + return err + } + + switch metadata.Method { + case partition.PartitionMethod_Hash, + partition.PartitionMethod_Key, + partition.PartitionMethod_LinearHash, + partition.PartitionMethod_LinearKey: + return moerr.NewNotSupportedNoCtx("add partition is not supported for hash/key partitioned table") + case partition.PartitionMethod_Range: + // TODO: check overlapping range + + case partition.PartitionMethod_List: + // TODO: check overlapping list values + } + + values := make([]partition.Partition, 0, len(partitions)) + n := len(metadata.Partitions) + for i, p := range partitions { + values = append(values, + partition.Partition{ + Name: p.Name.String(), + PartitionTableName: GetPartitionTableName(def.Name, p.Name.String()), + Position: uint32(i + n), + ExprStr: getExpr(p), + Expr: newTestValuesInExpr2(p.Name.String()), }, ) } - err = s.store.Delete( + return s.store.AddPartitions( + ctx, + def, + metadata, + values, + txnOp, + ) +} + +func (s *Service) DropPartitions( + ctx context.Context, + tableID uint64, + partitions []string, + txnOp client.TxnOperator, +) error { + metadata, ok, err := s.store.GetMetadata( + ctx, + tableID, + txnOp, + ) + if err != nil { + return err + } + if !ok { + return moerr.NewInternalError(ctx, fmt.Sprintf("table %d is not partitioned", tableID)) + } + + def, err := s.store.GetTableDef( + ctx, + tableID, + txnOp, + ) + if err != nil { + return err + } + + switch metadata.Method { + case partition.PartitionMethod_Hash, + partition.PartitionMethod_Key, + partition.PartitionMethod_LinearHash, + partition.PartitionMethod_LinearKey: + return moerr.NewNotSupportedNoCtx("drop partition is not supported for hash/key partitioned table") + case partition.PartitionMethod_Range: + case partition.PartitionMethod_List: + } + + return s.store.DropPartitions( ctx, + def, metadata, + partitions, txnOp, ) - if err == nil && txnOp == nil { - s.mu.Lock() - delete(s.mu.tables, tableID) - s.mu.Unlock() +} + +func (s *Service) TruncatePartitions( + ctx context.Context, + tableID uint64, + partitions []string, + txnOp client.TxnOperator, +) error { + metadata, ok, err := s.store.GetMetadata( + ctx, + tableID, + txnOp, + ) + if err != nil { + return err + } + if !ok { + return moerr.NewInternalError(ctx, fmt.Sprintf("table %d is not partitioned", tableID)) + } + + def, err := s.store.GetTableDef( + ctx, + tableID, + txnOp, + ) + if err != nil { + return err } - return err + + if len(partitions) == 0 { + for _, p := range metadata.Partitions { + partitions = append(partitions, p.Name) + } + } + + return s.store.TruncatePartitions( + ctx, + def, + metadata, + partitions, + txnOp, + ) +} + +func (s *Service) Delete( + ctx context.Context, + tableID uint64, + txnOp client.TxnOperator, +) error { + metadata, ok, err := s.store.GetMetadata( + ctx, + tableID, + txnOp, + ) + if err != nil { + return err + } + if !ok { + return nil + } + + return s.store.Delete( + ctx, + metadata, + txnOp, + ) } func (s *Service) GetPartitionMetadata( @@ -194,13 +390,7 @@ func (s *Service) readMetadata( tableID uint64, txnOp client.TxnOperator, ) (partition.PartitionMetadata, error) { - s.mu.RLock() - c, ok := s.mu.tables[tableID] - s.mu.RUnlock() - if ok { - return c.metadata, nil - } - + // TODO: use cache metadata, ok, err := s.store.GetMetadata( ctx, tableID, @@ -212,10 +402,6 @@ func (s *Service) readMetadata( if !ok { return partition.PartitionMetadata{}, nil } - - s.mu.Lock() - s.mu.tables[tableID] = newMetadataCache(metadata) - s.mu.Unlock() return metadata, nil } @@ -254,21 +440,97 @@ func (s *Service) getManualPartitions( return metadata, nil } -type metadataCache struct { - metadata partition.PartitionMetadata -} - -func newMetadataCache( - metadata partition.PartitionMetadata, -) metadataCache { - return metadataCache{ - metadata: metadata, - } -} - func GetPartitionTableName( tableName string, partitionName string, ) string { return "%!%" + partitionName + "%!%" + tableName } + +func getExpr(p *tree.Partition) string { + ctx := tree.NewFmtCtx( + dialect.MYSQL, + tree.WithQuoteIdentifier(), + tree.WithSingleQuoteString(), + ) + p.Values.Format(ctx) + return ctx.String() +} + +func newTestValuesInExpr2(col string) *plan.Expr { + return &plan.Expr{ + Typ: plan.Type{Id: 10}, + Expr: &plan.Expr_F{ + F: &plan.Function{ + Func: &plan.ObjectRef{ + ObjName: "in", + Obj: 506806140934, + }, + Args: []*plan.Expr{ + { + Typ: plan.Type{Id: 22}, + Expr: &plan.Expr_Col{ + Col: &plan.ColRef{RelPos: 1, ColPos: 0, Name: col}, + }, + }, + { + Typ: plan.Type{Id: 202}, + Expr: &plan.Expr_List{ + List: &plan.ExprList{ + List: []*plan.Expr{ + { + Typ: plan.Type{Id: 22}, + Expr: &plan.Expr_F{ + F: &plan.Function{ + Func: &plan.ObjectRef{ + Obj: 90194313216, + ObjName: "cast", + }, + Args: []*plan.Expr{ + { + Typ: plan.Type{Id: 23}, + Expr: &plan.Expr_Lit{ + Lit: &plan.Literal{ + Value: &plan.Literal_I64Val{I64Val: 1}, + }, + }, + }, + { + Typ: plan.Type{Id: 23}, + Expr: &plan.Expr_T{T: &plan.TargetType{}}, + }, + }, + }, + }, + }, + { + Typ: plan.Type{Id: 22}, + Expr: &plan.Expr_F{ + F: &plan.Function{ + Func: &plan.ObjectRef{ObjName: "cast", Obj: 90194313216}, + Args: []*plan.Expr{ + { + Typ: plan.Type{Id: 23}, + Expr: &plan.Expr_Lit{ + Lit: &plan.Literal{ + Value: &plan.Literal_I64Val{I64Val: 2}, + }, + }, + }, + { + Typ: plan.Type{Id: 22}, + Expr: &plan.Expr_T{T: &plan.TargetType{}}, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } +} diff --git a/pkg/partitionservice/service_list.go b/pkg/partitionservice/service_list.go index 351fe040521ba..b445d6f7bb4f4 100644 --- a/pkg/partitionservice/service_list.go +++ b/pkg/partitionservice/service_list.go @@ -41,14 +41,6 @@ func (s *Service) getMetadataByListType( def, desc, partition.PartitionMethod_List, - func(p *tree.Partition) string { - ctx := tree.NewFmtCtx( - dialect.MYSQL, - tree.WithQuoteIdentifier(), - tree.WithSingleQuoteString(), - ) - p.Values.Format(ctx) - return ctx.String() - }, + getExpr, ) } diff --git a/pkg/partitionservice/service_range.go b/pkg/partitionservice/service_range.go index ecd2601d02d1c..cbd338629a754 100644 --- a/pkg/partitionservice/service_range.go +++ b/pkg/partitionservice/service_range.go @@ -41,14 +41,6 @@ func (s *Service) getMetadataByRangeType( def, desc, partition.PartitionMethod_Range, - func(p *tree.Partition) string { - ctx := tree.NewFmtCtx( - dialect.MYSQL, - tree.WithQuoteIdentifier(), - tree.WithSingleQuoteString(), - ) - p.Values.Format(ctx) - return ctx.String() - }, + getExpr, ) } diff --git a/pkg/partitionservice/service_test.go b/pkg/partitionservice/service_test.go index 980abf5dbf57f..82d84f452e669 100644 --- a/pkg/partitionservice/service_test.go +++ b/pkg/partitionservice/service_test.go @@ -51,13 +51,103 @@ func TestDelete(t *testing.T) { assert.NoError(t, s.Create(ctx, tableID, stmt, txnOp)) require.NoError(t, txnOp.Commit(ctx)) - require.Empty(t, s.mu.tables) require.NoError(t, s.Delete(ctx, tableID, nil)) }, ) } +func TestRedefine(t *testing.T) { + runTestPartitionServiceTest( + func( + ctx context.Context, + txnOp client.TxnOperator, + s *Service, + store PartitionStorage, + ) { + require.Error(t, s.Redefine(ctx, 1, nil, txnOp)) + }, + ) +} + +func TestRenamePartition(t *testing.T) { + runTestPartitionServiceTest( + func( + ctx context.Context, + txnOp client.TxnOperator, + s *Service, + store PartitionStorage, + ) { + require.Error(t, s.Rename(ctx, 1, "old", "new", txnOp)) + }, + ) +} + +func TestAddPartitions(t *testing.T) { + runTestPartitionServiceTest( + func( + ctx context.Context, + txnOp client.TxnOperator, + s *Service, + store PartitionStorage, + ) { + require.Error(t, s.AddPartitions(ctx, 1, nil, txnOp)) + + tableID := uint64(1) + num := uint64(2) + columns := []string{"a"} + def := newTestTablePartitionDefine(1, columns, []types.T{types.T_int8}, num, partition.PartitionMethod_Hash) + memStore := store.(*memStorage) + memStore.addUncommittedTable(def) + + stmt := newTestHashOption(t, columns[0], num) + assert.NoError(t, s.Create(ctx, tableID, stmt, txnOp)) + require.NoError(t, txnOp.Commit(ctx)) + + require.Error(t, s.AddPartitions(ctx, tableID, nil, txnOp)) + }, + ) +} + +func TestDropPartitions(t *testing.T) { + runTestPartitionServiceTest( + func( + ctx context.Context, + txnOp client.TxnOperator, + s *Service, + store PartitionStorage, + ) { + require.Error(t, s.DropPartitions(ctx, 1, nil, txnOp)) + + tableID := uint64(1) + num := uint64(2) + columns := []string{"a"} + def := newTestTablePartitionDefine(1, columns, []types.T{types.T_int8}, num, partition.PartitionMethod_Hash) + memStore := store.(*memStorage) + memStore.addUncommittedTable(def) + + stmt := newTestHashOption(t, columns[0], num) + assert.NoError(t, s.Create(ctx, tableID, stmt, txnOp)) + require.NoError(t, txnOp.Commit(ctx)) + + require.Error(t, s.DropPartitions(ctx, tableID, nil, txnOp)) + }, + ) +} + +func TestTruncatePartitions(t *testing.T) { + runTestPartitionServiceTest( + func( + ctx context.Context, + txnOp client.TxnOperator, + s *Service, + store PartitionStorage, + ) { + require.Error(t, s.TruncatePartitions(ctx, 1, nil, txnOp)) + }, + ) +} + func TestIterResult(t *testing.T) { res := PruneResult{ batches: make([]*batch.Batch, 10), diff --git a/pkg/partitionservice/storage.go b/pkg/partitionservice/storage.go index ef7c422f4fe05..e83c1a2afafef 100644 --- a/pkg/partitionservice/storage.go +++ b/pkg/partitionservice/storage.go @@ -19,13 +19,16 @@ import ( "fmt" "strings" + "github.com/google/uuid" "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/defines" "github.com/matrixorigin/matrixone/pkg/pb/api" "github.com/matrixorigin/matrixone/pkg/pb/partition" "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/sql/features" + "github.com/matrixorigin/matrixone/pkg/sql/parsers" "github.com/matrixorigin/matrixone/pkg/sql/parsers/dialect" "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" "github.com/matrixorigin/matrixone/pkg/txn/client" @@ -275,6 +278,435 @@ func (s *Storage) Create( ) } +func (s *Storage) Redefine( + ctx context.Context, + def *plan.TableDef, + options *tree.PartitionOption, + metadata partition.PartitionMetadata, + txnOp client.TxnOperator, +) error { + accountID, err := defines.GetAccountId(ctx) + if err != nil { + return err + } + + opts := executor.Options{}. + WithTxn(txnOp). + WithAccountID(accountID) + if txnOp != nil { + opts = opts.WithDisableIncrStatement() + } + + return s.exec.ExecTxn( + ctx, + func(txn executor.TxnExecutor) error { + v, _ := parsers.ParseOne( + ctx, + dialect.MYSQL, + def.Createsql, + 1, + ) + tmp := uuid.NewString() + stmt := v.(*tree.CreateTable) + stmt.PartitionOption = options + table := stmt.Table + stmt.Table = *tree.NewTableName( + tree.Identifier(tmp), + table.ObjectNamePrefix, + table.AtTsExpr, + ) + sql := tree.StringWithOpts( + stmt, + dialect.MYSQL, + tree.WithQuoteIdentifier(), + tree.WithSingleQuoteString(), + ) + + // 1. create a temporary table + txn.Use(def.DbName) + rs, err := txn.Exec( + sql, + executor.StatementOption{}, + ) + if err != nil { + return err + } + rs.Close() + + // 2. select data into new temporary table + sql = fmt.Sprintf("insert into `%s` select * from `%s`", + tmp, + def.Name, + ) + rs, err = txn.Exec( + sql, + executor.StatementOption{}, + ) + if err != nil { + return err + } + rs.Close() + + // 3. drop old table + sql = fmt.Sprintf("drop table `%s`", def.Name) + rs, err = txn.Exec( + sql, + executor.StatementOption{}, + ) + if err != nil { + return err + } + rs.Close() + + // 4. rename tmp to old table name + sql = fmt.Sprintf("rename table `%s` to `%s`", tmp, def.Name) + rs, err = txn.Exec( + sql, + executor.StatementOption{}, + ) + if err != nil { + return err + } + rs.Close() + + return nil + }, + opts, + ) +} + +func (s *Storage) Rename( + ctx context.Context, + def *plan.TableDef, + oldName, newName string, + metadata partition.PartitionMetadata, + txnOp client.TxnOperator, +) error { + accountID, err := defines.GetAccountId(ctx) + if err != nil { + return err + } + + opts := executor.Options{}. + WithTxn(txnOp). + WithAccountID(accountID) + + if txnOp != nil { + opts = opts.WithDisableIncrStatement() + } + + return s.exec.ExecTxn( + ctx, + func(txn executor.TxnExecutor) error { + for _, p := range metadata.Partitions { + txn.Use(metadata.DatabaseName) + res, err := txn.Exec( + fmt.Sprintf( + "rename table `%s` to `%s`", + p.PartitionTableName, + GetPartitionTableName(newName, p.Name), + ), + executor.StatementOption{}. + WithIgnoreForeignKey(). + WithIgnorePublish(), + ) + if err != nil { + return err + } + res.Close() + + txn.Use(catalog.MO_CATALOG) + res, err = txn.Exec( + fmt.Sprintf("update %s set partition_table_name = '%s' where partition_table_name = '%s' and primary_table_id = %d", + catalog.MOPartitionTables, + GetPartitionTableName(newName, p.Name), + p.PartitionTableName, + metadata.TableID, + ), + executor.StatementOption{}, + ) + if err != nil { + return err + } + res.Close() + + res, err = txn.Exec( + fmt.Sprintf("update %s set table_name = '%s' where table_id = %d", + catalog.MOPartitionMetadata, + newName, + metadata.TableID, + ), + executor.StatementOption{}, + ) + if err != nil { + return err + } + res.Close() + } + return nil + }, + opts, + ) +} + +func (s *Storage) AddPartitions( + ctx context.Context, + def *plan.TableDef, + metadata partition.PartitionMetadata, + partitions []partition.Partition, + txnOp client.TxnOperator, +) error { + accountID, err := defines.GetAccountId(ctx) + if err != nil { + return err + } + + opts := executor.Options{}. + WithTxn(txnOp). + WithAccountID(accountID). + WithAdjustTableExtraFunc( + func(extra *api.SchemaExtra) error { + extra.ParentTableID = metadata.TableID + extra.FeatureFlag |= features.Partition + return nil + }, + ) + if txnOp != nil { + opts = opts.WithDisableIncrStatement() + } + + metadata.Partitions = append(metadata.Partitions, partitions...) + return s.exec.ExecTxn( + ctx, + func(txn executor.TxnExecutor) error { + stmt, _ := parsers.ParseOne( + ctx, + dialect.MYSQL, + def.Createsql, + 1, + ) + + err = s.updatePartitionCount( + metadata, + txn, + ) + if err != nil { + return err + } + + for _, p := range partitions { + err := s.createPartitionTable( + def, + stmt.(*tree.CreateTable), + metadata, + p, + txn, + ) + if err != nil { + return err + } + } + return nil + }, + opts, + ) +} + +func (s *Storage) DropPartitions( + ctx context.Context, + def *plan.TableDef, + metadata partition.PartitionMetadata, + partitions []string, + txnOp client.TxnOperator, +) error { + accountID, err := defines.GetAccountId(ctx) + if err != nil { + return err + } + + opts := executor.Options{}. + WithTxn(txnOp). + WithAccountID(accountID) + + if txnOp != nil { + opts = opts.WithDisableIncrStatement() + } + + whereCause := "" + tables := make([]string, 0, len(partitions)) + for _, p := range partitions { + found := false + for i, mp := range metadata.Partitions { + if p == mp.Name { + metadata.Partitions = append(metadata.Partitions[:i], metadata.Partitions[i+1:]...) + whereCause += fmt.Sprintf("'%s',", p) + tables = append(tables, mp.PartitionTableName) + found = true + break + } + } + if !found { + return moerr.NewInvalidInput(ctx, fmt.Sprintf("partition %s not found", p)) + } + } + + return s.exec.ExecTxn( + ctx, + func(txn executor.TxnExecutor) error { + txn.Use(catalog.MO_CATALOG) + err = s.updatePartitionCount( + metadata, + txn, + ) + if err != nil { + return err + } + + res, err := txn.Exec( + fmt.Sprintf("delete from %s where primary_table_id = %d and partition_name in (%s)", + catalog.MOPartitionTables, + metadata.TableID, + whereCause[:len(whereCause)-1], + ), + executor.StatementOption{}, + ) + if err != nil { + return err + } + res.Close() + + for i, p := range metadata.Partitions { + p.Position = uint32(i) + + res, err := txn.Exec( + fmt.Sprintf("update %s set partition_ordinal_position = %d where partition_id = %d and primary_table_id = %d", + catalog.MOPartitionTables, + p.Position, + p.PartitionID, + metadata.TableID, + ), + executor.StatementOption{}, + ) + if err != nil { + return err + } + res.Close() + } + + txn.Use(metadata.DatabaseName) + for _, name := range tables { + res, err = txn.Exec( + fmt.Sprintf( + "drop table `%s`", + name, + ), + executor.StatementOption{}. + WithIgnoreForeignKey(). + WithIgnorePublish(), + ) + if err != nil { + return err + } + res.Close() + } + return nil + }, + opts, + ) +} + +func (s *Storage) TruncatePartitions( + ctx context.Context, + def *plan.TableDef, + metadata partition.PartitionMetadata, + partitions []string, + txnOp client.TxnOperator, +) error { + accountID, err := defines.GetAccountId(ctx) + if err != nil { + return err + } + + opts := executor.Options{}. + WithTxn(txnOp). + WithAccountID(accountID). + WithAdjustTableExtraFunc( + func(extra *api.SchemaExtra) error { + extra.ParentTableID = def.TblId + extra.FeatureFlag |= features.Partition + return nil + }, + ) + + if txnOp != nil { + opts = opts.WithDisableIncrStatement() + } + + tables := make([]string, 0, len(partitions)) + for _, p := range partitions { + found := false + for _, mp := range metadata.Partitions { + if p == mp.Name { + tables = append(tables, mp.PartitionTableName) + found = true + break + } + } + if !found { + return moerr.NewInvalidInput(ctx, fmt.Sprintf("partition %s not found", p)) + } + } + + return s.exec.ExecTxn( + ctx, + func(txn executor.TxnExecutor) error { + for _, name := range tables { + txn.Use(metadata.DatabaseName) + res, err := txn.Exec( + fmt.Sprintf( + "truncate table `%s`", + name, + ), + executor.StatementOption{}. + WithIgnoreForeignKey(). + WithIgnorePublish(), + ) + if err != nil { + return err + } + res.Close() + + id, err := s.getTableIDByTableNameAndDatabaseName( + name, + metadata.DatabaseName, + txn, + ) + if err != nil { + return err + } + + txn.Use(catalog.MO_CATALOG) + res, err = txn.Exec( + fmt.Sprintf("update %s set partition_id = %d where partition_table_name = '%s' and primary_table_id = %d", + catalog.MOPartitionTables, + id, + name, + metadata.TableID, + ), + executor.StatementOption{}, + ) + if err != nil { + return err + } + res.Close() + } + + return nil + }, + opts, + ) +} + func (s *Storage) Delete( ctx context.Context, metadata partition.PartitionMetadata, @@ -330,7 +762,6 @@ func (s *Storage) Delete( p.PartitionTableName, ), executor.StatementOption{}. - WithIgnoreForeignKey(). WithIgnorePublish(). WithIgnoreForeignKey(), ) @@ -496,6 +927,34 @@ func (s *Storage) createPartitionMetadata( return nil } +func (s *Storage) updatePartitionCount( + metadata partition.PartitionMetadata, + txn executor.TxnExecutor, +) error { + txn.Use(catalog.MO_CATALOG) + + sql := fmt.Sprintf(` + update %s.%s + set partition_count = %d + where table_id = %d + `, + catalog.MO_CATALOG, + catalog.MOPartitionMetadata, + len(metadata.Partitions), + metadata.TableID, + ) + + res, err := txn.Exec( + sql, + executor.StatementOption{}, + ) + if err != nil { + return err + } + res.Close() + return nil +} + func getPartitionTableCreateSQL( stmt *tree.CreateTable, partition partition.Partition, diff --git a/pkg/partitionservice/storage_test.go b/pkg/partitionservice/storage_test.go index 1feea540c5e10..ea82e2514c89b 100644 --- a/pkg/partitionservice/storage_test.go +++ b/pkg/partitionservice/storage_test.go @@ -182,6 +182,56 @@ func (s *memStorage) nextTableID() uint64 { return atomic.AddUint64(&s.id, 1) } +func (s *memStorage) AddPartitions( + ctx context.Context, + def *plan.TableDef, + metadata partition.PartitionMetadata, + partitions []partition.Partition, + txnOp client.TxnOperator, +) error { + return nil +} + +func (s *memStorage) DropPartitions( + ctx context.Context, + def *plan.TableDef, + metadata partition.PartitionMetadata, + partitions []string, + txnOp client.TxnOperator, +) error { + return nil +} + +func (s *memStorage) TruncatePartitions( + ctx context.Context, + def *plan.TableDef, + metadata partition.PartitionMetadata, + partitions []string, + txnOp client.TxnOperator, +) error { + return nil +} + +func (s *memStorage) Rename( + ctx context.Context, + def *plan.TableDef, + oldName, newName string, + metadata partition.PartitionMetadata, + txnOp client.TxnOperator, +) error { + return nil +} + +func (s *memStorage) Redefine( + ctx context.Context, + def *plan.TableDef, + options *tree.PartitionOption, + metadata partition.PartitionMetadata, + txnOp client.TxnOperator, +) error { + return nil +} + type partitionTable struct { metadata partition.PartitionMetadata def *plan.TableDef diff --git a/pkg/partitionservice/types.go b/pkg/partitionservice/types.go index ef5f962898496..b99e937b52d34 100644 --- a/pkg/partitionservice/types.go +++ b/pkg/partitionservice/types.go @@ -55,7 +55,6 @@ var ( // PartitionService is used to maintaining the metadata of the partition table. type PartitionService interface { - // Create creates metadata of the partition table. Create( ctx context.Context, tableID uint64, @@ -63,12 +62,47 @@ type PartitionService interface { txnOp client.TxnOperator, ) error + Redefine( + ctx context.Context, + tableID uint64, + stmt *tree.PartitionOption, + txnOp client.TxnOperator, + ) error + + Rename( + ctx context.Context, + tableID uint64, + oldName, newName string, + txnOp client.TxnOperator, + ) error + Delete( ctx context.Context, tableID uint64, txnOp client.TxnOperator, ) error + AddPartitions( + ctx context.Context, + tableID uint64, + partitions []*tree.Partition, + txnOp client.TxnOperator, + ) error + + DropPartitions( + ctx context.Context, + tableID uint64, + partitions []string, + txnOp client.TxnOperator, + ) error + + TruncatePartitions( + ctx context.Context, + tableID uint64, + partitions []string, + txnOp client.TxnOperator, + ) error + GetPartitionMetadata( ctx context.Context, tableID uint64, @@ -95,6 +129,46 @@ type PartitionStorage interface { txnOp client.TxnOperator, ) error + Redefine( + ctx context.Context, + def *plan.TableDef, + options *tree.PartitionOption, + metadata partition.PartitionMetadata, + txnOp client.TxnOperator, + ) error + + Rename( + ctx context.Context, + def *plan.TableDef, + oldName, newName string, + metadata partition.PartitionMetadata, + txnOp client.TxnOperator, + ) error + + AddPartitions( + ctx context.Context, + def *plan.TableDef, + metadata partition.PartitionMetadata, + partitions []partition.Partition, + txnOp client.TxnOperator, + ) error + + DropPartitions( + ctx context.Context, + def *plan.TableDef, + metadata partition.PartitionMetadata, + partitions []string, + txnOp client.TxnOperator, + ) error + + TruncatePartitions( + ctx context.Context, + def *plan.TableDef, + metadata partition.PartitionMetadata, + partitions []string, + txnOp client.TxnOperator, + ) error + Delete( ctx context.Context, metadata partition.PartitionMetadata, diff --git a/pkg/pb/plan/plan.pb.go b/pkg/pb/plan/plan.pb.go index 592142252e724..e7c19dbb2b506 100644 --- a/pkg/pb/plan/plan.pb.go +++ b/pkg/pb/plan/plan.pb.go @@ -217,6 +217,37 @@ func (TableLockType) EnumDescriptor() ([]byte, []int) { return fileDescriptor_2d655ab2f7683c23, []int{6} } +type AlterPartitionType int32 + +const ( + AlterPartitionType_AddPartitionTables AlterPartitionType = 0 + AlterPartitionType_DropPartitionTables AlterPartitionType = 1 + AlterPartitionType_TruncatePartitionTables AlterPartitionType = 2 + AlterPartitionType_RedefinePartitionTables AlterPartitionType = 3 +) + +var AlterPartitionType_name = map[int32]string{ + 0: "AddPartitionTables", + 1: "DropPartitionTables", + 2: "TruncatePartitionTables", + 3: "RedefinePartitionTables", +} + +var AlterPartitionType_value = map[string]int32{ + "AddPartitionTables": 0, + "DropPartitionTables": 1, + "TruncatePartitionTables": 2, + "RedefinePartitionTables": 3, +} + +func (x AlterPartitionType) String() string { + return proto.EnumName(AlterPartitionType_name, int32(x)) +} + +func (AlterPartitionType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_2d655ab2f7683c23, []int{7} +} + type SubqueryRef_Type int32 const ( @@ -10545,13 +10576,14 @@ type AlterTable struct { DetectSqls []string `protobuf:"bytes,10,rep,name=detectSqls,proto3" json:"detectSqls,omitempty"` // alter table may insert fk records related to this table // into mo_foreign_keys - UpdateFkSqls []string `protobuf:"bytes,11,rep,name=updateFkSqls,proto3" json:"updateFkSqls,omitempty"` - RawSQL string `protobuf:"bytes,12,opt,name=RawSQL,proto3" json:"RawSQL,omitempty"` - Options *AlterCopyOpt `protobuf:"bytes,13,opt,name=Options,proto3" json:"Options,omitempty"` - AffectedCols []string `protobuf:"bytes,14,rep,name=AffectedCols,proto3" json:"AffectedCols,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + UpdateFkSqls []string `protobuf:"bytes,11,rep,name=updateFkSqls,proto3" json:"updateFkSqls,omitempty"` + RawSQL string `protobuf:"bytes,12,opt,name=RawSQL,proto3" json:"RawSQL,omitempty"` + Options *AlterCopyOpt `protobuf:"bytes,13,opt,name=Options,proto3" json:"Options,omitempty"` + AffectedCols []string `protobuf:"bytes,14,rep,name=AffectedCols,proto3" json:"AffectedCols,omitempty"` + AlterPartition *AlterPartitionOption `protobuf:"bytes,15,opt,name=alterPartition,proto3" json:"alterPartition,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *AlterTable) Reset() { *m = AlterTable{} } @@ -10685,6 +10717,13 @@ func (m *AlterTable) GetAffectedCols() []string { return nil } +func (m *AlterTable) GetAlterPartition() *AlterPartitionOption { + if m != nil { + return m.AlterPartition + } + return nil +} + type AlterTable_Action struct { // Types that are valid to be assigned to Action: // @@ -12927,6 +12966,53 @@ func (m *CloneTable) GetDstTableName() string { return "" } +type AlterPartitionOption struct { + AlterType AlterPartitionType `protobuf:"varint,1,opt,name=AlterType,proto3,enum=plan.AlterPartitionType" json:"AlterType,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *AlterPartitionOption) Reset() { *m = AlterPartitionOption{} } +func (m *AlterPartitionOption) String() string { return proto.CompactTextString(m) } +func (*AlterPartitionOption) ProtoMessage() {} +func (*AlterPartitionOption) Descriptor() ([]byte, []int) { + return fileDescriptor_2d655ab2f7683c23, []int{132} +} +func (m *AlterPartitionOption) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AlterPartitionOption) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_AlterPartitionOption.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *AlterPartitionOption) XXX_Merge(src proto.Message) { + xxx_messageInfo_AlterPartitionOption.Merge(m, src) +} +func (m *AlterPartitionOption) XXX_Size() int { + return m.ProtoSize() +} +func (m *AlterPartitionOption) XXX_DiscardUnknown() { + xxx_messageInfo_AlterPartitionOption.DiscardUnknown(m) +} + +var xxx_messageInfo_AlterPartitionOption proto.InternalMessageInfo + +func (m *AlterPartitionOption) GetAlterType() AlterPartitionType { + if m != nil { + return m.AlterType + } + return AlterPartitionType_AddPartitionTables +} + func init() { proto.RegisterEnum("plan.CompressType", CompressType_name, CompressType_value) proto.RegisterEnum("plan.ShuffleType", ShuffleType_name, ShuffleType_value) @@ -12935,6 +13021,7 @@ func init() { proto.RegisterEnum("plan.ExternType", ExternType_name, ExternType_value) proto.RegisterEnum("plan.TransationCompletionType", TransationCompletionType_name, TransationCompletionType_value) proto.RegisterEnum("plan.TableLockType", TableLockType_name, TableLockType_value) + proto.RegisterEnum("plan.AlterPartitionType", AlterPartitionType_name, AlterPartitionType_value) proto.RegisterEnum("plan.SubqueryRef_Type", SubqueryRef_Type_name, SubqueryRef_Type_value) proto.RegisterEnum("plan.Function_FuncFlag", Function_FuncFlag_name, Function_FuncFlag_value) proto.RegisterEnum("plan.ForeignKeyDef_RefAction", ForeignKeyDef_RefAction_name, ForeignKeyDef_RefAction_value) @@ -13096,791 +13183,798 @@ func init() { proto.RegisterType((*CreateCDC)(nil), "plan.CreateCDC") proto.RegisterType((*DropCDC)(nil), "plan.DropCDC") proto.RegisterType((*CloneTable)(nil), "plan.CloneTable") + proto.RegisterType((*AlterPartitionOption)(nil), "plan.AlterPartitionOption") } func init() { proto.RegisterFile("plan.proto", fileDescriptor_2d655ab2f7683c23) } var fileDescriptor_2d655ab2f7683c23 = []byte{ - // 12461 bytes of a gzipped FileDescriptorProto + // 12549 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0xbd, 0x5b, 0x8c, 0x1b, 0x49, - 0x96, 0x18, 0x5a, 0x2c, 0xbe, 0x0f, 0x1f, 0x95, 0x15, 0x2a, 0x49, 0x94, 0x5a, 0x2d, 0x95, 0x52, - 0x6a, 0xb5, 0x46, 0xdd, 0x2d, 0x75, 0x4b, 0xfd, 0x50, 0xcf, 0xce, 0xec, 0x0c, 0x8b, 0x64, 0xa9, - 0x38, 0x62, 0x91, 0x35, 0x49, 0x96, 0xd4, 0x3d, 0x8b, 0x45, 0x22, 0xc9, 0x4c, 0x56, 0x65, 0x57, - 0x32, 0x93, 0x9d, 0x99, 0x54, 0x55, 0x0d, 0xb0, 0xc0, 0xe0, 0x5e, 0x60, 0xf7, 0xee, 0xbd, 0x9f, - 0x17, 0xbb, 0xf6, 0x87, 0x17, 0x98, 0x5d, 0x60, 0x61, 0xc3, 0xb0, 0x01, 0xdb, 0x58, 0x63, 0xed, - 0x6f, 0xff, 0xac, 0x6d, 0x78, 0x61, 0xc0, 0x80, 0x0d, 0xdb, 0xc0, 0xda, 0x18, 0x7f, 0xfb, 0x05, - 0x1b, 0xf0, 0xa7, 0x8d, 0x73, 0x22, 0x32, 0x19, 0x49, 0xb2, 0x5a, 0xdd, 0x3d, 0xb3, 0x80, 0xfd, - 0x53, 0x95, 0x71, 0x1e, 0xf1, 0x8e, 0x13, 0xe7, 0x9c, 0x38, 0x11, 0x04, 0x98, 0x3a, 0x86, 0xfb, - 0x70, 0xea, 0x7b, 0xa1, 0xc7, 0x32, 0xf8, 0x7d, 0xfd, 0xbd, 0x23, 0x3b, 0x3c, 0x9e, 0x0d, 0x1f, - 0x8e, 0xbc, 0xc9, 0xa3, 0x23, 0xef, 0xc8, 0x7b, 0x44, 0xc8, 0xe1, 0x6c, 0x4c, 0x29, 0x4a, 0xd0, - 0x17, 0x67, 0xba, 0x0e, 0x8e, 0x37, 0x3a, 0x11, 0xdf, 0x1b, 0xa1, 0x3d, 0xb1, 0x82, 0xd0, 0x98, - 0x4c, 0x39, 0x40, 0xfd, 0xd3, 0x14, 0x64, 0x06, 0xe7, 0x53, 0x8b, 0x55, 0x61, 0xdd, 0x36, 0x6b, - 0xa9, 0xed, 0xd4, 0xfd, 0xac, 0xb6, 0x6e, 0x9b, 0x6c, 0x1b, 0x4a, 0xae, 0x17, 0x76, 0x67, 0x8e, - 0x63, 0x0c, 0x1d, 0xab, 0xb6, 0xbe, 0x9d, 0xba, 0x5f, 0xd0, 0x64, 0x10, 0x7b, 0x03, 0x8a, 0xc6, - 0x2c, 0xf4, 0x74, 0xdb, 0x1d, 0xf9, 0xb5, 0x34, 0xe1, 0x0b, 0x08, 0x68, 0xbb, 0x23, 0x9f, 0x6d, - 0x41, 0xf6, 0xd4, 0x36, 0xc3, 0xe3, 0x5a, 0x86, 0x72, 0xe4, 0x09, 0x84, 0x06, 0x23, 0xc3, 0xb1, - 0x6a, 0x59, 0x0e, 0xa5, 0x04, 0x42, 0x43, 0x2a, 0x24, 0xb7, 0x9d, 0xba, 0x5f, 0xd4, 0x78, 0x82, - 0xdd, 0x04, 0xb0, 0xdc, 0xd9, 0xe4, 0x95, 0xe1, 0xcc, 0xac, 0xa0, 0x96, 0x27, 0x94, 0x04, 0x51, - 0x7f, 0x00, 0xc5, 0x49, 0x70, 0xb4, 0x67, 0x19, 0xa6, 0xe5, 0xb3, 0xab, 0x90, 0x9f, 0x04, 0x47, - 0x7a, 0x68, 0x1c, 0x89, 0x26, 0xe4, 0x26, 0xc1, 0xd1, 0xc0, 0x38, 0x62, 0xd7, 0xa0, 0x40, 0x88, - 0xf3, 0x29, 0x6f, 0x43, 0x56, 0x43, 0x42, 0x6c, 0xb1, 0xfa, 0xbb, 0x39, 0xc8, 0x77, 0xec, 0xd0, - 0xf2, 0x0d, 0x87, 0x5d, 0x81, 0x9c, 0x1d, 0xb8, 0x33, 0xc7, 0x21, 0xf6, 0x82, 0x26, 0x52, 0xec, - 0x0a, 0x64, 0xed, 0xa7, 0xaf, 0x0c, 0x87, 0xf3, 0xee, 0xad, 0x69, 0x3c, 0xc9, 0x6a, 0x90, 0xb3, - 0x3f, 0xf8, 0x18, 0x11, 0x69, 0x81, 0x10, 0x69, 0xc2, 0x3c, 0x79, 0x8c, 0x98, 0x4c, 0x8c, 0xa1, - 0x34, 0x61, 0x3e, 0xfe, 0x10, 0x31, 0xd8, 0xfa, 0x34, 0x61, 0x28, 0x8d, 0xa5, 0xcc, 0xa8, 0x14, - 0xec, 0x80, 0x0a, 0x96, 0x32, 0x8b, 0x4a, 0x99, 0xf1, 0x52, 0xf2, 0x02, 0x21, 0xd2, 0x84, 0xe1, - 0xa5, 0x14, 0x62, 0x4c, 0x5c, 0xca, 0x8c, 0x97, 0x52, 0xdc, 0x4e, 0xdd, 0xcf, 0x10, 0x86, 0x97, - 0xb2, 0x05, 0x19, 0x13, 0xe1, 0xb0, 0x9d, 0xba, 0x9f, 0xda, 0x5b, 0xd3, 0x28, 0x85, 0xd0, 0x00, - 0xa1, 0x25, 0xec, 0x60, 0x84, 0x06, 0x02, 0x3a, 0x44, 0x68, 0x19, 0x7b, 0x03, 0xa1, 0x43, 0x01, - 0x1d, 0x23, 0xb4, 0xb2, 0x9d, 0xba, 0xbf, 0x8e, 0x50, 0x4c, 0xb1, 0xeb, 0x90, 0x37, 0x8d, 0xd0, - 0x42, 0x44, 0x55, 0x34, 0x39, 0x02, 0x20, 0x0e, 0x67, 0x1c, 0xe2, 0x36, 0x44, 0xa3, 0x23, 0x00, - 0x53, 0xa1, 0x84, 0x64, 0x11, 0x5e, 0x11, 0x78, 0x19, 0xc8, 0x3e, 0x82, 0xb2, 0x69, 0x8d, 0xec, - 0x89, 0xe1, 0xf0, 0x36, 0x6d, 0x6e, 0xa7, 0xee, 0x97, 0x1e, 0x6f, 0x3c, 0xa4, 0x35, 0x11, 0x63, - 0xf6, 0xd6, 0xb4, 0x04, 0x19, 0x7b, 0x0a, 0x15, 0x91, 0xfe, 0xe0, 0x31, 0x75, 0x2c, 0x23, 0x3e, - 0x25, 0xc1, 0xf7, 0xc1, 0xe3, 0xa7, 0x7b, 0x6b, 0x5a, 0x92, 0x90, 0xdd, 0x85, 0x72, 0xbc, 0x44, - 0x90, 0xf1, 0x92, 0xa8, 0x55, 0x02, 0x8a, 0xcd, 0xfa, 0x22, 0xf0, 0x5c, 0x24, 0xd8, 0x12, 0xfd, - 0x16, 0x01, 0xd8, 0x36, 0x80, 0x69, 0x8d, 0x8d, 0x99, 0x13, 0x22, 0xfa, 0xb2, 0xe8, 0x40, 0x09, - 0xc6, 0x6e, 0x42, 0x71, 0x36, 0xc5, 0x56, 0xbe, 0x30, 0x9c, 0xda, 0x15, 0x41, 0x30, 0x07, 0x61, - 0xee, 0x38, 0xcf, 0x11, 0x7b, 0x55, 0x8c, 0x6e, 0x04, 0xc0, 0xe1, 0x7d, 0x65, 0x8d, 0x10, 0x55, - 0x13, 0x05, 0x8b, 0x34, 0xae, 0x22, 0x3b, 0xd8, 0xb1, 0xdd, 0xda, 0x35, 0x9a, 0xc1, 0x3c, 0xc1, - 0x6e, 0x40, 0x3a, 0xf0, 0x47, 0xb5, 0xeb, 0xd4, 0x7e, 0xe0, 0xed, 0x6f, 0x9d, 0x4d, 0x7d, 0x0d, - 0xc1, 0x3b, 0x79, 0xc8, 0xd2, 0x6a, 0x52, 0x6f, 0x40, 0xe1, 0xc0, 0xf0, 0x8d, 0x89, 0x66, 0x8d, - 0x99, 0x02, 0xe9, 0xa9, 0x17, 0x88, 0x75, 0x84, 0x9f, 0x6a, 0x07, 0x72, 0x2f, 0x0c, 0x1f, 0x71, - 0x0c, 0x32, 0xae, 0x31, 0xb1, 0x08, 0x59, 0xd4, 0xe8, 0x1b, 0xd7, 0x4e, 0x70, 0x1e, 0x84, 0xd6, - 0x44, 0x08, 0x09, 0x91, 0x42, 0xf8, 0x91, 0xe3, 0x0d, 0xc5, 0x1a, 0x29, 0x68, 0x22, 0xa5, 0xfe, - 0x5f, 0x29, 0xc8, 0x35, 0x3c, 0x07, 0xb3, 0xbb, 0x0a, 0x79, 0xdf, 0x72, 0xf4, 0x79, 0x71, 0x39, - 0xdf, 0x72, 0x0e, 0xbc, 0x00, 0x11, 0x23, 0x8f, 0x23, 0xf8, 0xaa, 0xcd, 0x8d, 0x3c, 0x42, 0x44, - 0x15, 0x48, 0x4b, 0x15, 0xb8, 0x06, 0x85, 0x70, 0xe8, 0xe8, 0x04, 0xcf, 0x10, 0x3c, 0x1f, 0x0e, - 0x9d, 0x2e, 0xa2, 0xae, 0x42, 0xde, 0x1c, 0x72, 0x4c, 0x96, 0x30, 0x39, 0x73, 0x88, 0x08, 0xf5, - 0x53, 0x28, 0x6a, 0xc6, 0xa9, 0xa8, 0xc6, 0x65, 0xc8, 0x61, 0x06, 0x42, 0xfe, 0x65, 0xb4, 0x6c, - 0x38, 0x74, 0xda, 0x26, 0x82, 0xb1, 0x12, 0xb6, 0x49, 0x75, 0xc8, 0x68, 0xd9, 0x91, 0xe7, 0xb4, - 0x4d, 0x75, 0x00, 0xd0, 0xf0, 0x7c, 0xff, 0x5b, 0x37, 0x61, 0x0b, 0xb2, 0xa6, 0x35, 0x0d, 0x8f, - 0xb9, 0xe8, 0xd0, 0x78, 0x42, 0x7d, 0x00, 0x05, 0x1c, 0x97, 0x8e, 0x1d, 0x84, 0xec, 0x26, 0x64, - 0x1c, 0x3b, 0x08, 0x6b, 0xa9, 0xed, 0xf4, 0xc2, 0xa8, 0x11, 0x5c, 0xdd, 0x86, 0xc2, 0xbe, 0x71, - 0xf6, 0x02, 0x47, 0x0e, 0x73, 0xa3, 0x21, 0x14, 0x43, 0x22, 0xc6, 0xb3, 0x0c, 0x30, 0x30, 0xfc, - 0x23, 0x2b, 0x24, 0x49, 0xf7, 0xdf, 0x52, 0x50, 0xea, 0xcf, 0x86, 0x5f, 0xce, 0x2c, 0xff, 0x1c, - 0xeb, 0x7c, 0x1f, 0xd2, 0xe1, 0xf9, 0x94, 0x38, 0xaa, 0x8f, 0xaf, 0xf0, 0xec, 0x25, 0xfc, 0x43, - 0x64, 0xd2, 0x90, 0x04, 0x1b, 0xe1, 0x7a, 0xa6, 0x15, 0xf5, 0x41, 0x56, 0xcb, 0x61, 0xb2, 0x6d, - 0xe2, 0x76, 0xe1, 0x4d, 0xc5, 0x28, 0xac, 0x7b, 0x53, 0xb6, 0x0d, 0xd9, 0xd1, 0xb1, 0xed, 0x98, - 0x34, 0x00, 0xc9, 0x3a, 0x73, 0x04, 0x8e, 0x92, 0xef, 0x9d, 0xea, 0x81, 0xfd, 0xd3, 0x48, 0xfc, - 0xe7, 0x7d, 0xef, 0xb4, 0x6f, 0xff, 0xd4, 0x52, 0x07, 0x62, 0x0f, 0x02, 0xc8, 0xf5, 0x1b, 0xf5, - 0x4e, 0x5d, 0x53, 0xd6, 0xf0, 0xbb, 0xf5, 0x59, 0xbb, 0x3f, 0xe8, 0x2b, 0x29, 0x56, 0x05, 0xe8, - 0xf6, 0x06, 0xba, 0x48, 0xaf, 0xb3, 0x1c, 0xac, 0xb7, 0xbb, 0x4a, 0x1a, 0x69, 0x10, 0xde, 0xee, - 0x2a, 0x19, 0x96, 0x87, 0x74, 0xbd, 0xfb, 0xb9, 0x92, 0xa5, 0x8f, 0x4e, 0x47, 0xc9, 0xa9, 0x7f, - 0xbe, 0x0e, 0xc5, 0xde, 0xf0, 0x0b, 0x6b, 0x14, 0x62, 0x9b, 0x71, 0x96, 0x5a, 0xfe, 0x2b, 0xcb, - 0xa7, 0x66, 0xa7, 0x35, 0x91, 0xc2, 0x86, 0x98, 0x43, 0x6a, 0x5c, 0x5a, 0x5b, 0x37, 0x87, 0x44, - 0x37, 0x3a, 0xb6, 0x26, 0x06, 0x35, 0x0e, 0xe9, 0x28, 0x85, 0xab, 0xc2, 0x1b, 0x7e, 0x41, 0xcd, - 0x4b, 0x6b, 0xf8, 0xc9, 0x6e, 0x41, 0x89, 0xe7, 0x21, 0xcf, 0x2f, 0xe0, 0xa0, 0xc5, 0xc9, 0x97, - 0x93, 0x27, 0x1f, 0x71, 0x52, 0xae, 0x1c, 0x29, 0xf6, 0x36, 0x0e, 0xea, 0x8a, 0x19, 0xed, 0x0d, - 0xbf, 0xe0, 0xd8, 0x02, 0x9f, 0xd1, 0xde, 0xf0, 0x0b, 0x42, 0xbd, 0x03, 0x9b, 0xc1, 0x6c, 0x18, - 0x8c, 0x7c, 0x7b, 0x1a, 0xda, 0x9e, 0xcb, 0x69, 0x8a, 0x44, 0xa3, 0xc8, 0x08, 0x22, 0xbe, 0x0f, - 0x85, 0xe9, 0x6c, 0xa8, 0xdb, 0xee, 0xd8, 0x23, 0xb1, 0x5f, 0x7a, 0x5c, 0xe1, 0x03, 0x73, 0x30, - 0x1b, 0xb6, 0xdd, 0xb1, 0xa7, 0xe5, 0xa7, 0xfc, 0x83, 0xa9, 0x50, 0x71, 0xbd, 0x50, 0x47, 0x55, - 0x41, 0x9f, 0x58, 0xa1, 0x41, 0xfb, 0x01, 0xdf, 0xf0, 0x3b, 0xde, 0xe8, 0x64, 0xdf, 0x0a, 0x0d, - 0xf5, 0x1e, 0xe4, 0x05, 0x1f, 0xee, 0xfd, 0xa1, 0xe5, 0x1a, 0x6e, 0xa8, 0xc7, 0x4a, 0x43, 0x81, - 0x03, 0xda, 0xa6, 0xfa, 0x27, 0x29, 0x50, 0xfa, 0x52, 0x55, 0x90, 0x79, 0xa5, 0xe4, 0x78, 0x13, - 0xc0, 0x18, 0x8d, 0xbc, 0x19, 0xcf, 0x86, 0x4f, 0xb0, 0xa2, 0x80, 0xb4, 0x4d, 0xb9, 0xff, 0xd2, - 0x89, 0xfe, 0xbb, 0x0d, 0xe5, 0x88, 0x4f, 0x5a, 0xf4, 0x25, 0x01, 0x8b, 0x7a, 0x30, 0x98, 0x25, - 0x56, 0x7e, 0x3e, 0x98, 0x71, 0xee, 0x2b, 0x90, 0x23, 0x0d, 0x23, 0x88, 0x46, 0x85, 0xa7, 0xd4, - 0x7f, 0x95, 0x82, 0x4a, 0xdb, 0x35, 0xad, 0xb3, 0xfe, 0xc8, 0x70, 0xa3, 0x4e, 0xb1, 0x03, 0xdd, - 0x46, 0x98, 0x1e, 0x8c, 0x0c, 0x57, 0x28, 0x07, 0x25, 0x3b, 0x88, 0xe9, 0xb0, 0x0d, 0x9c, 0x80, - 0x8a, 0x5a, 0xa7, 0x1c, 0x8b, 0x04, 0xa1, 0xc2, 0xee, 0xc1, 0xc6, 0xd0, 0x72, 0x3c, 0xf7, 0x48, - 0x0f, 0x3d, 0x9d, 0x6b, 0x39, 0xbc, 0x2d, 0x15, 0x0e, 0x1e, 0x78, 0x03, 0xd2, 0x76, 0xb6, 0x20, - 0x3b, 0x35, 0xfc, 0x30, 0xa8, 0x65, 0xb6, 0xd3, 0xb8, 0x8c, 0x29, 0x81, 0xdd, 0x6c, 0x07, 0xfa, - 0xcc, 0xb5, 0xbf, 0x9c, 0xf1, 0x66, 0x14, 0xb4, 0x82, 0x1d, 0x1c, 0x52, 0x9a, 0xdd, 0x07, 0x85, - 0x97, 0x4c, 0xd9, 0xca, 0xf3, 0xac, 0x4a, 0x70, 0xca, 0x98, 0x84, 0xdd, 0xff, 0xbb, 0x0e, 0x85, - 0xdd, 0x99, 0x3b, 0xc2, 0xc1, 0x60, 0x77, 0x20, 0x33, 0x9e, 0xb9, 0x23, 0x6a, 0x4b, 0xbc, 0x95, - 0xc6, 0xeb, 0x44, 0x23, 0x24, 0x4a, 0x20, 0xc3, 0x3f, 0x42, 0xc9, 0xb5, 0x24, 0x81, 0x10, 0xae, - 0xfe, 0x83, 0x14, 0xcf, 0x71, 0xd7, 0x31, 0x8e, 0x58, 0x01, 0x32, 0xdd, 0x5e, 0xb7, 0xa5, 0xac, - 0xb1, 0x32, 0x14, 0xda, 0xdd, 0x41, 0x4b, 0xeb, 0xd6, 0x3b, 0x4a, 0x8a, 0x96, 0xf3, 0xa0, 0xbe, - 0xd3, 0x69, 0x29, 0xeb, 0x88, 0x79, 0xd1, 0xeb, 0xd4, 0x07, 0xed, 0x4e, 0x4b, 0xc9, 0x70, 0x8c, - 0xd6, 0x6e, 0x0c, 0x94, 0x02, 0x53, 0xa0, 0x7c, 0xa0, 0xf5, 0x9a, 0x87, 0x8d, 0x96, 0xde, 0x3d, - 0xec, 0x74, 0x14, 0x85, 0x5d, 0x82, 0x8d, 0x18, 0xd2, 0xe3, 0xc0, 0x6d, 0x64, 0x79, 0x51, 0xd7, - 0xea, 0xda, 0x33, 0xe5, 0x87, 0xac, 0x00, 0xe9, 0xfa, 0xb3, 0x67, 0xca, 0xcf, 0x50, 0x32, 0x14, - 0x5f, 0xb6, 0xbb, 0xfa, 0x8b, 0x7a, 0xe7, 0xb0, 0xa5, 0xfc, 0x6c, 0x3d, 0x4a, 0xf7, 0xb4, 0x66, - 0x4b, 0x53, 0x7e, 0x96, 0x61, 0x9b, 0x50, 0xfe, 0x49, 0xaf, 0xdb, 0xda, 0xaf, 0x1f, 0x1c, 0x50, - 0x45, 0x7e, 0x56, 0x50, 0xff, 0x63, 0x06, 0x32, 0xd8, 0x12, 0xa6, 0xce, 0xa5, 0x60, 0xdc, 0x44, - 0x14, 0x43, 0x3b, 0x99, 0x3f, 0xfb, 0x8b, 0x5b, 0x6b, 0x5c, 0xfe, 0xdd, 0x86, 0xb4, 0x63, 0x87, - 0x34, 0xac, 0xf1, 0xda, 0x11, 0x3a, 0xe3, 0xde, 0x9a, 0x86, 0x38, 0x76, 0x13, 0x52, 0x5c, 0x10, - 0x96, 0x1e, 0x57, 0xc5, 0xe2, 0x12, 0x3b, 0xe9, 0xde, 0x9a, 0x96, 0x9a, 0xb2, 0x1b, 0x90, 0x7a, - 0x25, 0xa4, 0x62, 0x99, 0xe3, 0xf9, 0x5e, 0x8a, 0xd8, 0x57, 0x6c, 0x1b, 0xd2, 0x23, 0x8f, 0x6b, - 0x84, 0x31, 0x9e, 0xef, 0x2c, 0x98, 0xff, 0xc8, 0x73, 0xd8, 0x1d, 0x48, 0xfb, 0xc6, 0x29, 0x8d, - 0x6c, 0x3c, 0x5c, 0xf1, 0xd6, 0x85, 0x44, 0xbe, 0x71, 0x8a, 0x95, 0x18, 0x93, 0x1c, 0x89, 0x2b, - 0x11, 0x8d, 0x37, 0x16, 0x33, 0x66, 0xdb, 0x90, 0x3a, 0x25, 0x49, 0x12, 0x2b, 0x41, 0x2f, 0x6d, - 0xd7, 0xf4, 0x4e, 0xfb, 0x53, 0x6b, 0x84, 0x14, 0xa7, 0xec, 0x2d, 0x48, 0x07, 0xb3, 0x21, 0x49, - 0x92, 0xd2, 0xe3, 0xcd, 0xa5, 0x3d, 0x01, 0x0b, 0x0a, 0x66, 0x43, 0x76, 0x0f, 0x32, 0x23, 0xcf, - 0xf7, 0x85, 0x34, 0x51, 0xa2, 0x0a, 0x47, 0xdb, 0x21, 0x2a, 0x85, 0x88, 0xc7, 0x02, 0x43, 0x92, - 0x21, 0x31, 0xd1, 0x7c, 0x3f, 0xc2, 0x02, 0x43, 0x76, 0x57, 0x6c, 0x72, 0x65, 0xb9, 0xd6, 0xd1, - 0x16, 0x88, 0xf9, 0x20, 0x16, 0x07, 0x69, 0x62, 0x9c, 0x91, 0xc6, 0x19, 0x13, 0x45, 0x7b, 0x1f, - 0xd6, 0x69, 0x62, 0x9c, 0xb1, 0xbb, 0x90, 0x7e, 0x65, 0x8d, 0x48, 0xf9, 0x8c, 0x4b, 0x13, 0x83, - 0xf4, 0x82, 0x9a, 0x87, 0x68, 0x9a, 0xf7, 0x9e, 0x63, 0x92, 0x1e, 0x1a, 0x8f, 0xe5, 0xae, 0xe7, - 0x98, 0x2f, 0x68, 0x2c, 0x09, 0x89, 0x5b, 0xbe, 0x31, 0x3b, 0x43, 0x69, 0xa4, 0xf0, 0xcd, 0xd9, - 0x98, 0x9d, 0xb5, 0x4d, 0x14, 0xfe, 0xae, 0xf9, 0x8a, 0xb4, 0xcf, 0x94, 0x86, 0x9f, 0x68, 0x1e, - 0x05, 0x96, 0x63, 0x8d, 0x42, 0xfb, 0x95, 0x1d, 0x9e, 0x93, 0x7e, 0x99, 0xd2, 0x64, 0xd0, 0x4e, - 0x0e, 0x32, 0xd6, 0xd9, 0xd4, 0x57, 0xf7, 0x20, 0x2f, 0x4a, 0x59, 0xb2, 0xb1, 0xae, 0x41, 0xc1, - 0x0e, 0xf4, 0x91, 0xe7, 0x06, 0xa1, 0xd0, 0x9d, 0xf2, 0x76, 0xd0, 0xc0, 0x24, 0x8a, 0x4b, 0xd3, - 0x08, 0xf9, 0x26, 0x54, 0xd6, 0xe8, 0x5b, 0x7d, 0x0c, 0x30, 0x6f, 0x16, 0xd6, 0xc9, 0xb1, 0xdc, - 0x48, 0x4d, 0x73, 0x2c, 0x37, 0xe6, 0x59, 0x97, 0x78, 0xae, 0x41, 0x31, 0xd6, 0x8c, 0x59, 0x19, - 0x52, 0x86, 0xd8, 0xfe, 0x52, 0x86, 0x7a, 0x1f, 0x15, 0xd5, 0x48, 0xf7, 0x4d, 0xe2, 0x30, 0x15, - 0x6d, 0x8a, 0xa9, 0xa1, 0xfa, 0x3d, 0x28, 0x6b, 0x56, 0x30, 0x73, 0xc2, 0x86, 0xe7, 0x34, 0xad, - 0x31, 0x7b, 0x17, 0x20, 0x4e, 0x07, 0x42, 0x4b, 0x99, 0xcf, 0xdd, 0xa6, 0x35, 0xd6, 0x24, 0xbc, - 0xfa, 0x37, 0x32, 0xa4, 0xef, 0x35, 0xb9, 0xa2, 0x25, 0x34, 0xaa, 0x94, 0xa4, 0x51, 0xc5, 0x7b, - 0xc3, 0x7a, 0x52, 0xab, 0x3c, 0xb6, 0x4d, 0xd3, 0x72, 0x23, 0xed, 0x91, 0xa7, 0x70, 0xb0, 0x0d, - 0xe7, 0x88, 0x16, 0x54, 0xf5, 0x31, 0x8b, 0x0a, 0x9d, 0x4c, 0x7d, 0x2b, 0x08, 0xb8, 0xde, 0x62, - 0x38, 0x47, 0xd1, 0xda, 0xce, 0x7e, 0xd5, 0xda, 0xbe, 0x06, 0x05, 0xdc, 0xf2, 0xc8, 0xea, 0xcb, - 0xf1, 0xde, 0x17, 0xe6, 0x2d, 0x7b, 0x1b, 0xf2, 0x42, 0x5f, 0x17, 0x8b, 0x4a, 0x4c, 0x97, 0x26, - 0x07, 0x6a, 0x11, 0x96, 0xd5, 0x50, 0xc9, 0x9b, 0x4c, 0x2c, 0x37, 0x8c, 0xf6, 0x69, 0x91, 0x64, - 0xef, 0x40, 0xd1, 0x73, 0x75, 0xae, 0xd4, 0x8b, 0x55, 0x25, 0xa6, 0x6f, 0xcf, 0x3d, 0x24, 0xa8, - 0x56, 0xf0, 0xc4, 0x17, 0x56, 0xc5, 0xf1, 0x4e, 0xf5, 0x91, 0xe1, 0x9b, 0xb4, 0xb2, 0x0a, 0x5a, - 0xde, 0xf1, 0x4e, 0x1b, 0x86, 0x6f, 0x72, 0xbd, 0xe5, 0x4b, 0x77, 0x36, 0xa1, 0xd5, 0x54, 0xd1, - 0x44, 0x8a, 0xdd, 0x80, 0xe2, 0xc8, 0x99, 0x05, 0xa1, 0xe5, 0xef, 0x9c, 0x73, 0x33, 0x4d, 0x9b, - 0x03, 0xb0, 0x5e, 0x53, 0xdf, 0x9e, 0x18, 0xfe, 0x39, 0x2d, 0x9d, 0x82, 0x16, 0x25, 0x69, 0xa3, - 0x39, 0xb1, 0xcd, 0x33, 0x6e, 0xab, 0x69, 0x3c, 0x81, 0xf4, 0xc7, 0x64, 0x49, 0x07, 0xb4, 0x3e, - 0x0a, 0x5a, 0x94, 0xa4, 0x71, 0xa0, 0x4f, 0x5a, 0x11, 0x45, 0x4d, 0xa4, 0x12, 0x4a, 0xf7, 0xe6, - 0x85, 0x4a, 0x37, 0x5b, 0xd4, 0x7b, 0x3c, 0xdf, 0x3e, 0xb2, 0x85, 0xd6, 0x72, 0x89, 0xeb, 0x3d, - 0x1c, 0x44, 0x1b, 0xd5, 0x97, 0x90, 0x17, 0x5d, 0x8c, 0x3b, 0x10, 0x2e, 0x9f, 0xa4, 0x78, 0xe6, - 0x3b, 0x10, 0xc2, 0xd9, 0x1d, 0xa8, 0x88, 0xbc, 0x82, 0xd0, 0xb7, 0xdd, 0x23, 0x31, 0x79, 0xca, - 0x1c, 0xd8, 0x27, 0x18, 0x2a, 0x0a, 0x38, 0xbc, 0xba, 0x31, 0xb4, 0x1d, 0x5c, 0xa6, 0x69, 0xa1, - 0xd4, 0xcc, 0x1c, 0xa7, 0xce, 0x41, 0x6a, 0x0f, 0x0a, 0xd1, 0x80, 0xfc, 0x4a, 0xca, 0x54, 0x7f, - 0x3b, 0x05, 0x25, 0x52, 0x0f, 0x7a, 0xa4, 0xfc, 0xb0, 0x77, 0x81, 0x8d, 0x7c, 0xcb, 0x08, 0x2d, - 0xdd, 0x3a, 0x0b, 0x7d, 0x43, 0x28, 0x01, 0x5c, 0x93, 0x50, 0x38, 0xa6, 0x85, 0x08, 0xae, 0x07, - 0xdc, 0x82, 0xd2, 0xd4, 0xf0, 0x83, 0x48, 0xa9, 0xe4, 0x05, 0x00, 0x07, 0x09, 0x95, 0x4e, 0x71, - 0x8f, 0x7c, 0x63, 0xa2, 0x87, 0xde, 0x89, 0xe5, 0x72, 0x75, 0x9a, 0x1b, 0x12, 0x55, 0x82, 0x0f, - 0x10, 0x4c, 0x5a, 0xf5, 0xbf, 0x49, 0x41, 0xe5, 0x80, 0x8f, 0xfa, 0x73, 0xeb, 0xbc, 0xc9, 0xad, - 0xb7, 0x51, 0xb4, 0x62, 0x33, 0x1a, 0x7d, 0xb3, 0x9b, 0x50, 0x9a, 0x9e, 0x58, 0xe7, 0x7a, 0xc2, - 0xd2, 0x29, 0x22, 0xa8, 0x41, 0x6b, 0xf3, 0x3b, 0x90, 0xf3, 0xa8, 0x21, 0x62, 0x8f, 0x13, 0x5b, - 0x83, 0xd4, 0x42, 0x4d, 0x10, 0xa0, 0xba, 0x14, 0x67, 0x25, 0xeb, 0x65, 0x22, 0x33, 0xaa, 0xfe, - 0x16, 0x64, 0x11, 0x15, 0xd4, 0xb2, 0x5c, 0xcf, 0xa1, 0x04, 0x7b, 0x1f, 0x2a, 0x23, 0x6f, 0x32, - 0xd5, 0x23, 0x76, 0xb1, 0xdb, 0x25, 0x65, 0x4a, 0x09, 0x49, 0x0e, 0x78, 0x5e, 0xea, 0xef, 0xa7, - 0xa1, 0x40, 0x75, 0x10, 0x62, 0xc5, 0x36, 0xcf, 0x22, 0xb1, 0x52, 0xd4, 0xb2, 0xb6, 0x89, 0x52, - 0xfb, 0x35, 0xaa, 0x59, 0xac, 0x72, 0xa5, 0x65, 0x95, 0xeb, 0x0a, 0xe4, 0x84, 0xbe, 0x95, 0xe1, - 0x72, 0x67, 0x76, 0xb1, 0xb6, 0x95, 0x5d, 0xa5, 0x6d, 0xe1, 0x10, 0x72, 0x1a, 0xeb, 0x0c, 0xf7, - 0x37, 0x2e, 0x5a, 0x80, 0x40, 0x2d, 0x84, 0xc8, 0x42, 0x23, 0x9f, 0x14, 0x1a, 0x35, 0xc8, 0xbf, - 0xb2, 0x03, 0x1b, 0x27, 0x48, 0x81, 0x2f, 0x43, 0x91, 0x94, 0x86, 0xa1, 0xf8, 0xba, 0x61, 0x88, - 0x9b, 0x6d, 0x38, 0x47, 0x5c, 0xed, 0x8f, 0x9a, 0x5d, 0x77, 0x8e, 0x3c, 0xf6, 0x01, 0x5c, 0x9e, - 0xa3, 0x45, 0x6b, 0xc8, 0x3d, 0x46, 0x1e, 0x20, 0x8d, 0xc5, 0x94, 0xd4, 0x22, 0xb2, 0xcb, 0x1e, - 0xc0, 0xa6, 0xc4, 0x32, 0x45, 0xf5, 0x26, 0x20, 0x99, 0x53, 0xd4, 0x36, 0x62, 0x72, 0xd2, 0x7a, - 0x02, 0xf5, 0x1f, 0xaf, 0x43, 0x65, 0xd7, 0xf3, 0x2d, 0xfb, 0xc8, 0x9d, 0xcf, 0xba, 0x25, 0xcd, - 0x3f, 0x9a, 0x89, 0xeb, 0xd2, 0x4c, 0xbc, 0x05, 0xa5, 0x31, 0x67, 0xd4, 0xc3, 0x21, 0x77, 0x1a, - 0x64, 0x34, 0x10, 0xa0, 0xc1, 0xd0, 0xc1, 0xd5, 0x1c, 0x11, 0x10, 0x73, 0x86, 0x98, 0x23, 0x26, - 0xdc, 0x6b, 0xd8, 0x77, 0x49, 0xea, 0x9a, 0x96, 0x63, 0x85, 0x7c, 0x78, 0xaa, 0x8f, 0xdf, 0x8c, - 0x76, 0x7a, 0xa9, 0x4e, 0x0f, 0x35, 0x6b, 0x5c, 0x27, 0xf5, 0x08, 0x85, 0x70, 0x93, 0xc8, 0x05, - 0xaf, 0x90, 0xd8, 0xb9, 0xaf, 0xc9, 0xcb, 0x25, 0x87, 0x3a, 0x80, 0x62, 0x0c, 0x46, 0x5d, 0x57, - 0x6b, 0x09, 0xfd, 0x76, 0x8d, 0x95, 0x20, 0xdf, 0xa8, 0xf7, 0x1b, 0xf5, 0x66, 0x4b, 0x49, 0x21, - 0xaa, 0xdf, 0x1a, 0x70, 0x9d, 0x76, 0x9d, 0x6d, 0x40, 0x09, 0x53, 0xcd, 0xd6, 0x6e, 0xfd, 0xb0, - 0x33, 0x50, 0xd2, 0xac, 0x02, 0xc5, 0x6e, 0x4f, 0xaf, 0x37, 0x06, 0xed, 0x5e, 0x57, 0xc9, 0xa8, - 0x3f, 0x84, 0x42, 0xe3, 0xd8, 0x1a, 0x9d, 0x5c, 0xd4, 0x8b, 0x64, 0x74, 0x5b, 0xa3, 0x13, 0xa1, - 0x9f, 0x2e, 0x18, 0xdd, 0xd6, 0xe8, 0x44, 0x6d, 0x41, 0xf1, 0xc0, 0xf0, 0x43, 0x9b, 0xea, 0xf5, - 0x14, 0x2a, 0x71, 0xa2, 0x69, 0x8d, 0xa3, 0x9d, 0x9b, 0xc5, 0x5a, 0x6b, 0x8c, 0xd2, 0x92, 0x84, - 0xea, 0xbb, 0x50, 0x96, 0x01, 0xec, 0x06, 0xa4, 0x4d, 0x6b, 0xbc, 0x42, 0x4e, 0x22, 0x58, 0x7d, - 0x01, 0xe5, 0x46, 0xb4, 0x13, 0x5d, 0x54, 0xf5, 0xc7, 0x50, 0xa5, 0x15, 0x3f, 0x1a, 0x46, 0x4b, - 0x7e, 0x7d, 0xc5, 0x92, 0x2f, 0x23, 0x4d, 0x63, 0x28, 0xd6, 0xfc, 0x47, 0x50, 0x3a, 0xf0, 0xbd, - 0xa9, 0xe5, 0x87, 0x94, 0xad, 0x02, 0xe9, 0x13, 0xeb, 0x5c, 0xe4, 0x8a, 0x9f, 0x73, 0x5f, 0xc8, - 0xba, 0xec, 0x0b, 0x79, 0x0c, 0x85, 0x88, 0xed, 0x6b, 0xf3, 0xfc, 0x00, 0x45, 0x27, 0xf1, 0xd8, - 0x56, 0x80, 0x85, 0x3d, 0x04, 0x98, 0xc6, 0x00, 0xd1, 0x71, 0x91, 0xba, 0x2f, 0x32, 0xd7, 0x24, - 0x0a, 0xf5, 0x4d, 0xc8, 0xbf, 0xb0, 0xad, 0x53, 0xd1, 0xfc, 0x57, 0xb6, 0x75, 0x1a, 0x35, 0x1f, - 0xbf, 0xd5, 0xff, 0xaf, 0x08, 0x05, 0x5a, 0x5f, 0xcd, 0x8b, 0xdd, 0x4f, 0xdf, 0x44, 0x2b, 0xda, - 0x16, 0xeb, 0x29, 0xb3, 0x42, 0x17, 0xe3, 0xab, 0xeb, 0x4d, 0x00, 0x69, 0xad, 0x73, 0xc9, 0x55, - 0x0c, 0xe3, 0x25, 0x8e, 0xea, 0x04, 0xed, 0x45, 0xc1, 0x97, 0x8e, 0xb0, 0x22, 0xe7, 0x00, 0xf6, - 0x90, 0x6f, 0xf6, 0x64, 0x37, 0x72, 0x85, 0xe8, 0x52, 0xa4, 0xd4, 0x0f, 0x1d, 0x2b, 0x32, 0x35, - 0x48, 0x03, 0xc0, 0x04, 0xc9, 0x31, 0xcb, 0x0f, 0x50, 0x5c, 0x91, 0x7f, 0x5a, 0x8b, 0x92, 0xec, - 0x6d, 0xc8, 0xa0, 0x90, 0x17, 0xa6, 0xc1, 0xa5, 0xa8, 0x07, 0xa5, 0x5d, 0x4a, 0x23, 0x02, 0x76, - 0x1f, 0xf2, 0x24, 0x5a, 0x2c, 0x94, 0x34, 0x52, 0x6f, 0x47, 0x42, 0x5f, 0x8b, 0xd0, 0xec, 0x3b, - 0x90, 0x1d, 0x9f, 0x58, 0xe7, 0x41, 0xad, 0x42, 0x74, 0x97, 0x56, 0xac, 0x59, 0x8d, 0x53, 0xb0, - 0xbb, 0x50, 0xf5, 0xad, 0xb1, 0x4e, 0x0e, 0x29, 0x14, 0x32, 0x41, 0xad, 0x4a, 0x32, 0xa4, 0xec, - 0x5b, 0xe3, 0x06, 0x02, 0x07, 0x43, 0x27, 0x60, 0xf7, 0x20, 0x47, 0xab, 0x07, 0x75, 0x21, 0xa9, - 0xe4, 0x68, 0x29, 0x6a, 0x02, 0xcb, 0x3e, 0x00, 0x10, 0x1a, 0x97, 0x3e, 0x3c, 0x27, 0x47, 0x6e, - 0xbc, 0x98, 0xe4, 0xf9, 0x2f, 0xeb, 0x65, 0x6f, 0x43, 0x16, 0x27, 0x49, 0x50, 0xbb, 0x4a, 0x39, - 0x6f, 0x26, 0x67, 0x10, 0xd5, 0x94, 0xf0, 0xec, 0x3e, 0x14, 0x70, 0xa2, 0xe8, 0x38, 0x1c, 0x35, - 0x59, 0x05, 0x15, 0xb3, 0x0a, 0x77, 0x06, 0xeb, 0xb4, 0xff, 0xa5, 0xc3, 0x1e, 0x40, 0xc6, 0xc4, - 0xc5, 0x7c, 0x8d, 0x72, 0xbc, 0x22, 0x8d, 0x0b, 0x0a, 0xab, 0xa6, 0x35, 0x26, 0xad, 0x98, 0x68, - 0xd8, 0x1e, 0x54, 0x71, 0x1a, 0x3d, 0xa6, 0xcd, 0x1e, 0xbb, 0xaf, 0x76, 0x9d, 0xb8, 0x6e, 0x2f, - 0x70, 0x75, 0x05, 0x11, 0x75, 0x76, 0xcb, 0x0d, 0xfd, 0x73, 0xad, 0xe2, 0xca, 0x30, 0x76, 0x1d, - 0x4d, 0x97, 0x8e, 0x37, 0x3a, 0xb1, 0xcc, 0xda, 0x1b, 0x91, 0x63, 0x82, 0xa7, 0xd9, 0xa7, 0x50, - 0xa1, 0x89, 0x85, 0x49, 0x2c, 0xbc, 0x76, 0x83, 0x84, 0xa9, 0x3c, 0x65, 0x22, 0x94, 0x96, 0xa4, - 0x44, 0x11, 0x6f, 0x07, 0x7a, 0x68, 0x4d, 0xa6, 0x9e, 0x8f, 0xca, 0xeb, 0x9b, 0x91, 0xc3, 0x65, - 0x10, 0x81, 0x70, 0x23, 0x8e, 0x8f, 0x9d, 0x74, 0x6f, 0x3c, 0x0e, 0xac, 0xb0, 0x76, 0x93, 0xd6, - 0x4d, 0x35, 0x3a, 0x7d, 0xea, 0x11, 0x94, 0x36, 0xc2, 0x40, 0x37, 0xcf, 0x5d, 0x63, 0x62, 0x8f, - 0x6a, 0xb7, 0xb8, 0x8e, 0x6c, 0x07, 0x4d, 0x0e, 0x90, 0xd5, 0xd4, 0xed, 0x84, 0x9a, 0x7a, 0x09, - 0xb2, 0xe6, 0x10, 0x97, 0xe3, 0x6d, 0xca, 0x36, 0x63, 0x0e, 0xdb, 0x26, 0x7b, 0x0f, 0x8a, 0xd3, - 0x48, 0x04, 0xd6, 0x54, 0xd9, 0x18, 0x8f, 0x25, 0xa3, 0x36, 0xa7, 0x40, 0xfb, 0x70, 0xd7, 0x32, - 0xc2, 0x99, 0x6f, 0xed, 0x3a, 0xc6, 0x51, 0xed, 0x0e, 0xe5, 0x24, 0x83, 0xae, 0x3f, 0x23, 0x5d, - 0x97, 0x5a, 0xfd, 0xd1, 0x82, 0x70, 0x49, 0x2c, 0x0d, 0x49, 0x0a, 0xed, 0xad, 0xc9, 0x32, 0x66, - 0x27, 0x4b, 0x52, 0xf8, 0xfa, 0x0f, 0x81, 0x2d, 0x8f, 0xd7, 0xeb, 0x24, 0x5d, 0x56, 0x48, 0xba, - 0xef, 0xae, 0x3f, 0x4d, 0xa9, 0x2f, 0xa0, 0x92, 0x58, 0xc8, 0x2b, 0x25, 0x36, 0x57, 0x97, 0x8c, - 0x89, 0x30, 0x2f, 0x79, 0x42, 0x78, 0xa8, 0x02, 0xdb, 0x3d, 0x12, 0x9e, 0x2d, 0x9a, 0x08, 0x7d, - 0x4a, 0xab, 0x7f, 0x9e, 0x86, 0xf2, 0x9e, 0x11, 0x1c, 0xef, 0x1b, 0xd3, 0x7e, 0x68, 0x84, 0x01, - 0x0e, 0xef, 0xb1, 0x11, 0x1c, 0x4f, 0x8c, 0x29, 0x57, 0x5c, 0x53, 0xdc, 0x6c, 0x16, 0x30, 0xd4, - 0x5a, 0x71, 0x62, 0x61, 0xb2, 0xe7, 0x1e, 0x3c, 0x17, 0x36, 0x71, 0x9c, 0x46, 0xb1, 0x12, 0x1c, - 0xcf, 0xc6, 0xe3, 0xb8, 0xa8, 0x28, 0xc9, 0xee, 0x42, 0x45, 0x7c, 0x92, 0xd6, 0x7a, 0x26, 0x8e, - 0x1d, 0x93, 0x40, 0xf6, 0x04, 0x4a, 0x02, 0x30, 0x88, 0x84, 0x60, 0x35, 0xf6, 0x75, 0xcc, 0x11, - 0x9a, 0x4c, 0xc5, 0x7e, 0x0c, 0x97, 0xa5, 0xe4, 0xae, 0xe7, 0xef, 0xcf, 0x9c, 0xd0, 0x6e, 0x74, - 0x85, 0x8a, 0xf0, 0xc6, 0x12, 0xfb, 0x9c, 0x44, 0x5b, 0xcd, 0x99, 0xac, 0xed, 0xbe, 0xed, 0x92, - 0x4c, 0x4d, 0x6b, 0x49, 0xe0, 0x02, 0x95, 0x71, 0x46, 0xa2, 0x34, 0x49, 0x65, 0x9c, 0xe1, 0x62, - 0x13, 0x80, 0x7d, 0x2b, 0x3c, 0xf6, 0x4c, 0xd2, 0x0f, 0xe3, 0xc5, 0xd6, 0x97, 0x51, 0x5a, 0x92, - 0x12, 0xbb, 0x13, 0x2d, 0xa1, 0x91, 0x1b, 0x92, 0x96, 0x98, 0xd6, 0xa2, 0x24, 0x6e, 0x33, 0xbe, - 0xe1, 0x1e, 0x59, 0x41, 0xad, 0xb4, 0x9d, 0xbe, 0x9f, 0xd2, 0x44, 0x4a, 0xfd, 0xe3, 0x75, 0xc8, - 0xf2, 0x91, 0x7c, 0x03, 0x8a, 0x43, 0x72, 0x16, 0xa3, 0x65, 0x2a, 0x1c, 0xc0, 0x04, 0xe8, 0xce, - 0x26, 0x5c, 0xbb, 0x13, 0x3e, 0x8d, 0x94, 0x46, 0xdf, 0x98, 0xa5, 0x37, 0x0b, 0xb1, 0xac, 0x34, - 0x41, 0x45, 0x0a, 0x2b, 0xe1, 0x7b, 0xa7, 0x34, 0x1b, 0x32, 0x84, 0x88, 0x92, 0xe4, 0x63, 0xa6, - 0x1d, 0x0b, 0x99, 0xb2, 0x84, 0x2b, 0x10, 0xa0, 0xe1, 0x86, 0x8b, 0xfe, 0x97, 0xdc, 0x92, 0xff, - 0x85, 0xdd, 0x04, 0xd4, 0x1d, 0x47, 0x56, 0xcf, 0xb5, 0x1a, 0x5d, 0xea, 0xe1, 0x82, 0x26, 0x41, - 0x70, 0x81, 0x98, 0xde, 0x94, 0x3a, 0x35, 0xab, 0xe1, 0x27, 0xfb, 0x38, 0x9e, 0x9d, 0xd4, 0x46, - 0xa1, 0x69, 0x0b, 0x89, 0x2e, 0xcf, 0x63, 0x2d, 0x41, 0x87, 0x39, 0xa1, 0x98, 0xe6, 0x9a, 0x36, - 0x7e, 0xaa, 0x2d, 0x00, 0xcd, 0x3b, 0x0d, 0xac, 0x90, 0x1c, 0x8d, 0x57, 0xa9, 0x89, 0x89, 0x23, - 0x22, 0xef, 0xf4, 0xc0, 0x0b, 0x62, 0x7b, 0x73, 0x7d, 0xb5, 0xbd, 0xa9, 0x3e, 0x82, 0x3c, 0xee, - 0xe1, 0x46, 0x68, 0xb0, 0xbb, 0xc2, 0xb7, 0xc3, 0x35, 0x0f, 0xe1, 0xe4, 0x9a, 0x97, 0x21, 0xbc, - 0x3d, 0x9d, 0xa8, 0x5c, 0xe2, 0xb9, 0x2d, 0x99, 0x7b, 0xf1, 0xfe, 0x21, 0x32, 0x14, 0x5a, 0xc1, - 0x1b, 0x50, 0xc4, 0xaa, 0x91, 0xdf, 0x5c, 0xc8, 0x85, 0x82, 0xef, 0x9d, 0x36, 0x30, 0xad, 0xfe, - 0xdb, 0x14, 0x94, 0x7a, 0xbe, 0x89, 0x1b, 0x57, 0x7f, 0x6a, 0x8d, 0x5e, 0x6b, 0x1e, 0xa3, 0x0e, - 0xe1, 0x39, 0x8e, 0x41, 0x22, 0x52, 0x98, 0x5b, 0x31, 0x80, 0x7d, 0x00, 0x99, 0x31, 0x8a, 0xc2, - 0xb4, 0xac, 0x59, 0x4b, 0xd9, 0x47, 0xdf, 0x28, 0x1c, 0x35, 0x22, 0x55, 0x7f, 0x23, 0x2e, 0x9f, - 0xfc, 0xcc, 0xb2, 0x77, 0x79, 0x8d, 0xce, 0x79, 0xfa, 0x0d, 0x25, 0xc5, 0x0a, 0x90, 0x69, 0xb6, - 0xfa, 0x0d, 0xae, 0x4f, 0xa3, 0x66, 0xdd, 0xd7, 0x77, 0xdb, 0x5a, 0x7f, 0xa0, 0x64, 0xe8, 0xe0, - 0x88, 0x00, 0x9d, 0x7a, 0x7f, 0xa0, 0x14, 0x18, 0x40, 0xee, 0xb0, 0xdb, 0xfe, 0xf1, 0x61, 0x4b, - 0x51, 0xd4, 0x7f, 0x91, 0x02, 0x98, 0x3b, 0x41, 0xd9, 0x3b, 0x50, 0x3a, 0xa5, 0x94, 0x2e, 0x79, - 0xc7, 0xe5, 0x36, 0x02, 0x47, 0x93, 0x7e, 0xf3, 0x1e, 0x94, 0x63, 0x51, 0x8f, 0x7b, 0xff, 0xb2, - 0x9b, 0xbc, 0x14, 0xe3, 0x77, 0xce, 0xd9, 0xbb, 0x50, 0xf0, 0xb0, 0x1d, 0x48, 0x9a, 0x96, 0x37, - 0x7e, 0xa9, 0xf9, 0x5a, 0xde, 0xe3, 0x09, 0xd4, 0x11, 0xc6, 0x7e, 0x64, 0x3e, 0xc7, 0xa4, 0xbb, - 0x08, 0x6a, 0x38, 0xc6, 0x2c, 0xb0, 0x34, 0x8e, 0x8f, 0xa5, 0x74, 0x76, 0x2e, 0xa5, 0xd5, 0x9f, - 0x40, 0xb5, 0x6f, 0x4c, 0xa6, 0x5c, 0x96, 0x53, 0xc3, 0x18, 0x64, 0x70, 0x4e, 0x88, 0xa9, 0x47, - 0xdf, 0xb8, 0xe8, 0x0e, 0x2c, 0x7f, 0x64, 0xb9, 0xd1, 0x1a, 0x8d, 0x92, 0x28, 0x7e, 0x0f, 0x51, - 0x9a, 0x6b, 0xde, 0x69, 0x24, 0xce, 0xa3, 0xb4, 0xfa, 0xb7, 0x52, 0x50, 0x92, 0xaa, 0xc1, 0x1e, - 0x41, 0x86, 0x94, 0xc9, 0x94, 0x2c, 0x08, 0x25, 0x02, 0xfe, 0xcd, 0xd5, 0x0f, 0x24, 0x64, 0xf7, - 0x20, 0x1b, 0x84, 0x86, 0x1f, 0xf9, 0xd3, 0x15, 0x89, 0x63, 0xc7, 0x9b, 0xb9, 0xa6, 0xc6, 0xd1, - 0x4c, 0x85, 0xb4, 0xe5, 0x9a, 0xc2, 0xe1, 0xb0, 0x4c, 0x85, 0x48, 0x75, 0x1b, 0x8a, 0x71, 0xf6, - 0x38, 0x05, 0xb4, 0xde, 0xcb, 0xbe, 0xb2, 0xc6, 0x8a, 0x90, 0xd5, 0xea, 0xdd, 0x67, 0x2d, 0x25, - 0xa5, 0xfe, 0x49, 0x0a, 0x60, 0xce, 0xc5, 0x1e, 0x26, 0x6a, 0x7b, 0x7d, 0x31, 0xd7, 0x87, 0xf4, - 0x57, 0xaa, 0xec, 0x0d, 0x28, 0xce, 0x5c, 0x02, 0x5a, 0xa6, 0xd8, 0x89, 0xe6, 0x00, 0xb4, 0x80, - 0xa2, 0xe8, 0x8f, 0x05, 0x0b, 0xe8, 0x95, 0xe1, 0xa8, 0xdf, 0x85, 0x62, 0x9c, 0x1d, 0x1a, 0x75, - 0xbb, 0xbd, 0x4e, 0xa7, 0xf7, 0xb2, 0xdd, 0x7d, 0xa6, 0xac, 0x61, 0xf2, 0x40, 0x6b, 0x35, 0x5a, - 0x4d, 0x4c, 0xa6, 0x70, 0xce, 0x36, 0x0e, 0x35, 0xad, 0xd5, 0x1d, 0xe8, 0x5a, 0xef, 0xa5, 0xb2, - 0xae, 0xfe, 0xdf, 0x19, 0xd8, 0xec, 0xb9, 0xcd, 0xd9, 0xd4, 0xb1, 0x47, 0x46, 0x68, 0x3d, 0xb7, - 0xce, 0x1b, 0xe1, 0x19, 0xee, 0xbe, 0x46, 0x18, 0xfa, 0x7c, 0x31, 0x17, 0x35, 0x9e, 0xe0, 0x4e, - 0x89, 0xc0, 0xf2, 0x43, 0xf2, 0xb9, 0xc8, 0xab, 0xb8, 0xca, 0xe1, 0x0d, 0xcf, 0xa1, 0xb5, 0xcc, - 0xbe, 0x0f, 0x97, 0xb9, 0x23, 0x83, 0x53, 0xa2, 0x02, 0xab, 0xd3, 0x62, 0x4e, 0x2f, 0x4d, 0x5d, - 0xc6, 0x09, 0x91, 0x15, 0xc9, 0x48, 0x84, 0xdd, 0x82, 0xd2, 0x9c, 0x3d, 0x3a, 0xa4, 0x82, 0x98, - 0x90, 0x6a, 0x82, 0x86, 0x77, 0x54, 0x6b, 0xdd, 0x36, 0xcf, 0xc8, 0xc5, 0x93, 0xd5, 0xaa, 0xde, - 0xbc, 0x31, 0xb8, 0x09, 0x7f, 0x06, 0x9b, 0x09, 0x4a, 0xaa, 0x45, 0x8e, 0x6a, 0xf1, 0x6e, 0xe4, - 0x20, 0x5d, 0x68, 0xbd, 0x0c, 0xc1, 0xea, 0x70, 0x8d, 0x74, 0xc3, 0x4b, 0x42, 0x85, 0x2e, 0x62, - 0x1f, 0xb9, 0x9e, 0x6f, 0x09, 0x81, 0x5f, 0xb0, 0x83, 0x36, 0xa5, 0xe7, 0xf6, 0x8f, 0x74, 0xa8, - 0xca, 0xf7, 0x97, 0xe8, 0xbc, 0x90, 0xa3, 0x6d, 0xbe, 0x83, 0x66, 0xb4, 0x3c, 0xa5, 0xdb, 0x26, - 0xbb, 0x23, 0xd4, 0x59, 0x3d, 0x32, 0x69, 0x80, 0x4c, 0x9a, 0x32, 0x01, 0x5f, 0x70, 0xd8, 0xf5, - 0x2e, 0x6c, 0xad, 0xaa, 0xe4, 0x0a, 0x35, 0x6c, 0x5b, 0x56, 0xc3, 0x16, 0x8c, 0xf6, 0xb9, 0x4a, - 0xf6, 0x77, 0x52, 0x50, 0x6e, 0x5a, 0xe6, 0x6c, 0xfa, 0x23, 0xcf, 0x76, 0x71, 0x02, 0x7c, 0x08, - 0x65, 0xcf, 0x31, 0x69, 0xf4, 0xa4, 0xd8, 0x80, 0xc4, 0x89, 0x91, 0x70, 0x6e, 0x83, 0xe7, 0x98, - 0x0d, 0xcf, 0xa1, 0x48, 0x82, 0xf7, 0xe0, 0x12, 0x77, 0x68, 0x08, 0xff, 0xde, 0x19, 0x67, 0x5e, - 0xa7, 0x91, 0x51, 0x38, 0x8a, 0x2b, 0x47, 0x44, 0xfe, 0x6b, 0xb0, 0x25, 0x91, 0xe3, 0xc8, 0x70, - 0xfa, 0xe5, 0x49, 0xb2, 0x19, 0xf3, 0x46, 0x47, 0x36, 0xea, 0xef, 0xac, 0x43, 0x91, 0xbb, 0x43, - 0xb0, 0xbe, 0xf7, 0x21, 0xef, 0x0d, 0xbf, 0xd0, 0xfd, 0xd8, 0x4d, 0xb0, 0x74, 0xd2, 0x98, 0xf3, - 0x86, 0x5f, 0x68, 0xd6, 0x98, 0xbd, 0x13, 0xed, 0xf3, 0xa6, 0x35, 0x16, 0x9d, 0x52, 0x4d, 0xda, - 0x23, 0x62, 0xdf, 0x47, 0x5b, 0xf9, 0x09, 0x94, 0xe6, 0x33, 0x3e, 0xa8, 0xe5, 0x2f, 0xee, 0x85, - 0x78, 0x01, 0x04, 0xc8, 0xc4, 0x5d, 0x42, 0x9c, 0xa9, 0x70, 0x31, 0x13, 0x27, 0x23, 0xa6, 0x4f, - 0xa1, 0x3a, 0x97, 0xf1, 0xc4, 0x57, 0xbc, 0x90, 0xaf, 0x12, 0x53, 0xd2, 0x89, 0xc7, 0xdf, 0x4d, - 0x41, 0xb1, 0xcd, 0x8b, 0x0f, 0xcf, 0xd8, 0x6d, 0x48, 0x7f, 0x45, 0x2f, 0x20, 0x8e, 0x3d, 0x80, - 0x4d, 0xc3, 0x34, 0x75, 0x63, 0x3c, 0xb6, 0x46, 0xa1, 0x65, 0xea, 0xa8, 0x02, 0x09, 0x99, 0xb3, - 0x61, 0x98, 0x66, 0x5d, 0xc0, 0x49, 0x76, 0xe3, 0x9a, 0x0f, 0xf4, 0xc8, 0xf0, 0x9c, 0x1f, 0x29, - 0x17, 0xb4, 0xaa, 0x1d, 0x08, 0xbb, 0x93, 0xfb, 0x92, 0x13, 0x1d, 0x9b, 0xf9, 0xea, 0x8e, 0x55, - 0x7f, 0x6f, 0x1d, 0x40, 0xb3, 0xa6, 0x8e, 0x31, 0xb2, 0xfe, 0x8f, 0xa9, 0x34, 0x8a, 0xa5, 0x78, - 0x60, 0x5d, 0x33, 0x0a, 0xc1, 0x88, 0x06, 0xd1, 0x35, 0xd9, 0x0f, 0xe1, 0x4d, 0xdf, 0x3a, 0xf5, - 0xed, 0xd0, 0xd2, 0xc7, 0xbe, 0x37, 0xd1, 0x13, 0x92, 0x07, 0x17, 0x66, 0x91, 0x2a, 0x71, 0x4d, - 0x10, 0xed, 0xfa, 0xde, 0x24, 0x29, 0x7d, 0xd4, 0xff, 0x54, 0x80, 0x52, 0xdd, 0x35, 0x9c, 0xf3, - 0x9f, 0x5a, 0x14, 0x13, 0x40, 0xde, 0xd5, 0xe9, 0x2c, 0xe4, 0xcd, 0xe5, 0x07, 0x66, 0x45, 0x82, - 0x50, 0x43, 0x6f, 0x41, 0xc9, 0x9b, 0x85, 0x31, 0x9e, 0x1f, 0xa1, 0x01, 0x07, 0x11, 0x41, 0xcc, - 0x1f, 0x7b, 0xee, 0x23, 0x7e, 0x32, 0x7f, 0xe6, 0xfc, 0xb1, 0x4a, 0x1c, 0xf3, 0x13, 0x01, 0x4a, - 0x23, 0x7b, 0x42, 0x0d, 0x0e, 0x66, 0x13, 0x8b, 0x37, 0x3a, 0xcd, 0xe3, 0xd3, 0x1a, 0x02, 0x86, - 0xb9, 0x4c, 0xac, 0x89, 0xe7, 0x9f, 0xf3, 0x5c, 0x72, 0x3c, 0x17, 0x0e, 0xa2, 0x5c, 0xde, 0x05, - 0x76, 0x6a, 0xd8, 0xa1, 0x9e, 0xcc, 0x8a, 0x9b, 0x21, 0x0a, 0x62, 0x06, 0x72, 0x76, 0x57, 0x20, - 0x67, 0xda, 0xc1, 0x49, 0xbb, 0x27, 0x4c, 0x10, 0x91, 0xc2, 0xb6, 0x04, 0x23, 0x03, 0x35, 0xa0, - 0xd0, 0xe2, 0xea, 0x72, 0x5a, 0x2b, 0x22, 0x64, 0x07, 0x01, 0xb8, 0x83, 0xba, 0x56, 0x78, 0xea, - 0xf9, 0xc8, 0xc9, 0x2d, 0x8c, 0x39, 0x00, 0x35, 0x0d, 0x24, 0xc5, 0x82, 0xc8, 0x1b, 0x94, 0xd6, - 0xe2, 0x34, 0xea, 0xee, 0x7c, 0xf9, 0x12, 0xb6, 0xcc, 0xab, 0x3f, 0x87, 0xb0, 0xbb, 0x50, 0xa5, - 0xea, 0x93, 0x05, 0x82, 0x6d, 0xa0, 0x53, 0xae, 0xb4, 0x56, 0x46, 0x28, 0x39, 0x13, 0x90, 0xea, - 0x53, 0xb8, 0x96, 0x68, 0x9f, 0x6e, 0xf8, 0xbe, 0x71, 0xae, 0x4f, 0x8c, 0x2f, 0x3c, 0x9f, 0x1c, - 0x3f, 0x69, 0xed, 0x8a, 0xdc, 0x6d, 0x75, 0x44, 0xef, 0x23, 0xf6, 0x42, 0x56, 0xdb, 0xf5, 0x7c, - 0xf2, 0x0a, 0xad, 0x64, 0x45, 0x2c, 0xb9, 0x30, 0x68, 0x80, 0xc9, 0x1c, 0x0a, 0x78, 0x5c, 0xa3, - 0x56, 0x22, 0xd8, 0x0e, 0x81, 0xd0, 0x20, 0x08, 0x9e, 0x70, 0xc9, 0xba, 0x29, 0x82, 0x8c, 0x9e, - 0x90, 0xfc, 0xe5, 0x88, 0x63, 0xcb, 0x30, 0xe9, 0xe4, 0x8c, 0x10, 0x7b, 0x96, 0x41, 0xe7, 0xd2, - 0xc1, 0x13, 0x7d, 0x3a, 0x0b, 0x79, 0x40, 0xa2, 0x96, 0x0d, 0x9e, 0x1c, 0xcc, 0x42, 0x01, 0x3e, - 0xb2, 0x42, 0x0a, 0x43, 0x24, 0xf0, 0x33, 0x2b, 0xc4, 0x8d, 0x30, 0x78, 0x12, 0x79, 0xc1, 0x2f, - 0x8b, 0xbe, 0x7d, 0x22, 0xdc, 0xdc, 0x2a, 0x54, 0x62, 0xa4, 0x3e, 0x99, 0xf1, 0x08, 0xc4, 0xb4, - 0x56, 0x8a, 0x08, 0xf6, 0x67, 0x0e, 0x0e, 0xec, 0xc8, 0x18, 0x1d, 0x5b, 0xba, 0x8f, 0x55, 0xb9, - 0xca, 0x87, 0x8e, 0x20, 0x1a, 0xd6, 0xe6, 0x0d, 0xe0, 0x09, 0xfd, 0xd8, 0x0e, 0xc9, 0x3b, 0x95, - 0xd6, 0x0a, 0x04, 0xd8, 0xb3, 0x43, 0x14, 0x0b, 0x1c, 0x29, 0x66, 0x20, 0x65, 0x71, 0x8d, 0x88, - 0x36, 0x08, 0xb1, 0x4f, 0x70, 0xca, 0xe8, 0x3e, 0x28, 0x09, 0x5a, 0xcc, 0xef, 0x3a, 0x91, 0x56, - 0x25, 0x52, 0xcc, 0xf5, 0x1e, 0x70, 0x66, 0x1d, 0xa7, 0x1e, 0xcf, 0xf3, 0x0d, 0x6e, 0x0e, 0x13, - 0xb8, 0x69, 0x07, 0x27, 0x94, 0xe3, 0x5d, 0xa8, 0x4a, 0x74, 0x98, 0xdf, 0x0d, 0x3e, 0x33, 0x62, - 0xb2, 0x44, 0x1d, 0x7d, 0x6b, 0xe2, 0x85, 0xa2, 0x99, 0x6f, 0x4a, 0x75, 0xd4, 0x08, 0x9e, 0xac, - 0xa3, 0xa0, 0xc5, 0x3c, 0x6f, 0x4a, 0x75, 0xe4, 0xa4, 0x98, 0xeb, 0x6d, 0x28, 0xa3, 0x14, 0x09, - 0x2d, 0x97, 0x2f, 0xfe, 0x5b, 0xbc, 0x63, 0x05, 0x8c, 0x56, 0xff, 0x6d, 0x28, 0xf3, 0x9e, 0x17, - 0xe2, 0x72, 0x9b, 0x93, 0x08, 0x18, 0x92, 0xa8, 0x3f, 0x4b, 0xc1, 0xf5, 0x1e, 0x1d, 0x28, 0x92, - 0xc0, 0xdb, 0xb7, 0x82, 0xc0, 0x38, 0xb2, 0x76, 0x3d, 0x7f, 0x77, 0xf6, 0xd3, 0x9f, 0x9e, 0xb3, - 0xfb, 0xb0, 0x71, 0x60, 0xf8, 0x96, 0x1b, 0xc6, 0x07, 0x4e, 0x42, 0xb9, 0x58, 0x04, 0xb3, 0xa7, - 0xa0, 0x70, 0xd0, 0x61, 0xac, 0xa6, 0x09, 0x43, 0x25, 0xe9, 0x1f, 0x5e, 0xa2, 0x52, 0xff, 0xc7, - 0x36, 0x64, 0xba, 0x9e, 0x69, 0xb1, 0xf7, 0xa1, 0x48, 0xe1, 0x7f, 0x92, 0xe2, 0x2c, 0x1c, 0x0b, - 0x88, 0xa6, 0x3f, 0xa4, 0x31, 0x17, 0x5c, 0xf1, 0x75, 0x71, 0xc0, 0xe0, 0x6d, 0xd2, 0xfd, 0xe9, - 0xb4, 0x0d, 0x65, 0x7a, 0x49, 0xf8, 0x27, 0xc8, 0x9c, 0xe6, 0x18, 0x94, 0x08, 0xe4, 0x99, 0xf5, - 0x2d, 0x97, 0x34, 0xcc, 0xac, 0x16, 0xa7, 0xc9, 0xe2, 0xf2, 0x3d, 0xdc, 0x7f, 0xf8, 0xba, 0xc9, - 0xae, 0xb0, 0xb8, 0x38, 0x9e, 0x16, 0xd2, 0xfb, 0x50, 0xfc, 0xc2, 0xb3, 0x5d, 0x5e, 0xf1, 0xdc, - 0x52, 0xc5, 0x51, 0xa9, 0xe2, 0x15, 0xff, 0x42, 0x7c, 0xb1, 0x3b, 0x90, 0xf7, 0x5c, 0x9e, 0x77, - 0x7e, 0x29, 0xef, 0x9c, 0xe7, 0x76, 0x78, 0x34, 0x4a, 0xc5, 0x0e, 0x74, 0xdf, 0x3e, 0x3a, 0x0e, - 0x75, 0xe4, 0x14, 0xa7, 0x74, 0x25, 0x3b, 0xd0, 0x10, 0x86, 0xd9, 0xa2, 0x21, 0x39, 0xb6, 0x1d, - 0xdc, 0xe6, 0x28, 0xb3, 0xe2, 0x52, 0x66, 0xc0, 0xd1, 0x94, 0xe1, 0x5b, 0x50, 0x38, 0xf2, 0xbd, - 0xd9, 0x14, 0x2d, 0x43, 0x58, 0xa2, 0xcc, 0x13, 0x6e, 0xe7, 0x1c, 0x85, 0x3e, 0x7d, 0xda, 0xee, - 0x91, 0x4e, 0x46, 0x74, 0x69, 0x3b, 0x7d, 0xbf, 0xa0, 0x95, 0x23, 0x20, 0x99, 0xc7, 0x6f, 0x41, - 0xc1, 0x38, 0x3a, 0xd2, 0x45, 0x50, 0xcd, 0x52, 0x5e, 0xc6, 0xd1, 0x11, 0x15, 0xf9, 0x10, 0x2a, - 0xa7, 0xb6, 0xab, 0x07, 0x53, 0x6b, 0xc4, 0x69, 0x2b, 0xcb, 0x5d, 0x79, 0x6a, 0xbb, 0x68, 0x3b, - 0x12, 0xbd, 0x6c, 0xbc, 0x56, 0x5f, 0x6b, 0xbc, 0x6e, 0x43, 0xd6, 0xb1, 0x27, 0x76, 0x28, 0xc2, - 0x6c, 0x12, 0xda, 0x2d, 0x21, 0x98, 0x0a, 0x39, 0xe1, 0xb5, 0x55, 0x96, 0x48, 0x04, 0x26, 0xa9, - 0x04, 0x6c, 0xbe, 0x46, 0x09, 0x90, 0x34, 0x4d, 0xf6, 0xd5, 0x9a, 0xe6, 0x47, 0xa4, 0xd2, 0x59, - 0x6e, 0xa8, 0x47, 0x0c, 0x97, 0x56, 0x33, 0x94, 0x39, 0x59, 0x8f, 0xb3, 0x7d, 0x00, 0x25, 0x9f, - 0xbc, 0x2a, 0x3a, 0xb9, 0x60, 0xb6, 0x64, 0xb3, 0x74, 0xee, 0x6e, 0xd1, 0xc0, 0x9f, 0xbb, 0x5e, - 0xea, 0xb0, 0x31, 0x0f, 0x1b, 0xe4, 0xf1, 0x97, 0x97, 0x65, 0xb7, 0x6e, 0x22, 0xce, 0x30, 0x52, - 0x22, 0xed, 0x44, 0xf0, 0xe1, 0x1d, 0xa8, 0xf0, 0x80, 0x01, 0x7e, 0xac, 0x1b, 0x90, 0x9c, 0x2e, - 0x6a, 0x65, 0x02, 0xf2, 0x23, 0xdf, 0x80, 0x3d, 0x04, 0x88, 0x14, 0xa0, 0xf0, 0x8c, 0x04, 0x75, - 0xdc, 0x1a, 0x2e, 0xcd, 0x1b, 0xe1, 0x99, 0x56, 0x34, 0xa3, 0x4f, 0x94, 0x3f, 0x43, 0xdb, 0x35, - 0x71, 0x1e, 0x85, 0xc6, 0x51, 0x50, 0xab, 0xd1, 0x32, 0x2b, 0x09, 0xd8, 0xc0, 0x38, 0x0a, 0xd0, - 0xd0, 0x30, 0xb8, 0xbe, 0xc3, 0xeb, 0x7d, 0x4d, 0xf6, 0x42, 0x48, 0x9a, 0x90, 0x56, 0x32, 0x24, - 0xb5, 0xe8, 0x13, 0x60, 0xd1, 0x09, 0x91, 0x64, 0x37, 0x5c, 0x5f, 0x9a, 0x5a, 0x1b, 0xe2, 0x88, - 0x28, 0x8e, 0x75, 0xfe, 0x04, 0x2a, 0x49, 0xb5, 0xf0, 0xc6, 0x8a, 0x73, 0x14, 0x1a, 0x75, 0xad, - 0x3c, 0x92, 0x15, 0xc5, 0x3b, 0x3c, 0x62, 0x95, 0x64, 0x30, 0x31, 0xf2, 0xb3, 0x82, 0xb2, 0xeb, - 0x85, 0x8d, 0x08, 0x86, 0xfd, 0x13, 0x99, 0x0b, 0xe1, 0x19, 0x89, 0xed, 0xb8, 0x7f, 0x62, 0x05, - 0x1d, 0xd5, 0xaf, 0x48, 0x57, 0xc7, 0xa1, 0xe6, 0x4a, 0x30, 0x31, 0xdc, 0x4a, 0x0c, 0x75, 0xac, - 0x1d, 0x6b, 0xe0, 0xcf, 0x35, 0xe5, 0x5b, 0x50, 0x0a, 0xbc, 0x99, 0x3f, 0xb2, 0xf4, 0x20, 0xb4, - 0xa6, 0xb5, 0x6d, 0xea, 0x51, 0xe0, 0xa0, 0x7e, 0x68, 0x4d, 0xd9, 0x53, 0xa8, 0x4e, 0x7d, 0xec, - 0xcc, 0xb8, 0x1e, 0xaa, 0xdc, 0xc4, 0x03, 0xdf, 0x9a, 0x57, 0xa5, 0x3c, 0x95, 0x52, 0xec, 0x07, - 0xb0, 0x29, 0x71, 0xce, 0x4e, 0x88, 0xf9, 0x0e, 0x31, 0x6f, 0x2d, 0x30, 0x1f, 0x9e, 0x20, 0x7b, - 0x75, 0x9a, 0x48, 0xb3, 0xfa, 0x82, 0x55, 0x8e, 0x1a, 0xef, 0x5d, 0xe2, 0xbf, 0x7a, 0x81, 0xa9, - 0x9d, 0x30, 0xd7, 0x9f, 0xf3, 0x53, 0x83, 0x76, 0xd0, 0x72, 0xcd, 0xda, 0x5b, 0xfc, 0x5a, 0x01, - 0x25, 0xd8, 0x13, 0x28, 0x73, 0xdd, 0x8b, 0x82, 0xfa, 0x82, 0xda, 0x3d, 0xd9, 0x2d, 0x49, 0x0a, - 0x18, 0x21, 0xb4, 0x92, 0x13, 0x7f, 0x07, 0xec, 0x63, 0xd8, 0xe4, 0x3e, 0x63, 0x59, 0x3e, 0xbe, - 0xbd, 0x3c, 0x45, 0x88, 0x68, 0x77, 0x2e, 0x24, 0x35, 0xb8, 0xe6, 0xcf, 0x5c, 0xd2, 0xc7, 0x04, - 0xe7, 0xd4, 0xf7, 0x86, 0x16, 0xe7, 0xbf, 0x4f, 0xfc, 0xa2, 0x39, 0x1a, 0x27, 0xe3, 0xbc, 0x24, - 0x98, 0xae, 0xf8, 0x32, 0xe8, 0x00, 0xf9, 0x2e, 0xc8, 0x73, 0x38, 0xb3, 0x1d, 0x93, 0xe7, 0xf9, - 0x9d, 0x6f, 0x92, 0xe7, 0x0e, 0xf2, 0x51, 0x9e, 0x0c, 0x32, 0xb3, 0x99, 0x6d, 0xd6, 0x1e, 0xf0, - 0xf8, 0x3b, 0xfc, 0x66, 0x6f, 0x41, 0xd5, 0xb7, 0x46, 0x33, 0x3f, 0xb0, 0x5f, 0x59, 0x7a, 0x60, - 0xbb, 0x27, 0xb5, 0x77, 0xa8, 0x1f, 0x2b, 0x31, 0xb4, 0x6f, 0xbb, 0x27, 0x38, 0xef, 0xac, 0xb3, - 0xd0, 0xf2, 0x5d, 0x1e, 0x67, 0xfc, 0xae, 0x3c, 0xef, 0x5a, 0x84, 0x40, 0xb9, 0xa0, 0x81, 0x15, - 0x7f, 0x2f, 0x4c, 0x8e, 0x80, 0x4f, 0x8e, 0x87, 0x5f, 0x6b, 0x72, 0xf4, 0x69, 0x72, 0xdc, 0x83, - 0x82, 0xed, 0x86, 0x96, 0xff, 0xca, 0x70, 0x6a, 0x8f, 0x96, 0x44, 0x71, 0x8c, 0x63, 0x77, 0x21, - 0x1f, 0x38, 0x36, 0xca, 0x87, 0xda, 0xfb, 0x4b, 0x64, 0x11, 0x8a, 0xdd, 0x87, 0x62, 0x7c, 0x45, - 0xa6, 0xf6, 0xc1, 0x12, 0xdd, 0x1c, 0xc9, 0x6e, 0x42, 0xe6, 0x14, 0x27, 0xd4, 0xe3, 0x65, 0x37, - 0x32, 0xc2, 0x71, 0xef, 0x1e, 0xdb, 0x8e, 0xc3, 0xf7, 0xee, 0x27, 0x4b, 0x7b, 0xf7, 0xae, 0xed, - 0x38, 0x7c, 0xef, 0x1e, 0x8b, 0x2f, 0xdc, 0xf9, 0x88, 0x03, 0x5b, 0xf2, 0xe1, 0xf2, 0xce, 0x87, - 0xb8, 0x17, 0x74, 0x99, 0xa8, 0x14, 0x90, 0x6f, 0x94, 0xbb, 0x78, 0x3f, 0x92, 0xfb, 0x2a, 0xe9, - 0x34, 0xd5, 0x20, 0x88, 0xd3, 0xa8, 0x0c, 0x0b, 0xcf, 0xb0, 0x6d, 0x9e, 0xd5, 0x3e, 0xe6, 0x51, - 0xea, 0x1c, 0xd2, 0x36, 0xcf, 0xd8, 0xfb, 0x50, 0x89, 0xc2, 0x40, 0xb0, 0xb8, 0xa0, 0xf6, 0xc9, - 0x52, 0x0d, 0x92, 0x04, 0xac, 0x09, 0xe5, 0x31, 0xea, 0x72, 0x13, 0xae, 0xda, 0xd5, 0x9e, 0x52, - 0x45, 0xb6, 0xa3, 0x5d, 0xf5, 0x22, 0xd5, 0x4f, 0x4b, 0x70, 0xb1, 0x87, 0xc0, 0xec, 0x31, 0x1f, - 0x4f, 0x34, 0x5b, 0xb9, 0xfa, 0x56, 0xfb, 0x94, 0x66, 0xd7, 0x0a, 0x0c, 0x1d, 0x14, 0x59, 0xae, - 0xa9, 0x4f, 0x02, 0xa1, 0x23, 0x7c, 0x97, 0xea, 0x29, 0xa4, 0x61, 0x7c, 0x95, 0x4e, 0x6c, 0x46, - 0x25, 0xa4, 0xdd, 0x0f, 0xb8, 0xca, 0xf0, 0x29, 0xe0, 0x74, 0x7d, 0x35, 0x67, 0xfd, 0xb5, 0xaf, - 0x64, 0x45, 0xda, 0x88, 0xf5, 0x29, 0x54, 0x4d, 0xcb, 0x9c, 0x4d, 0x49, 0x5d, 0xa2, 0x29, 0xfa, - 0x3d, 0x59, 0xf8, 0xc9, 0x2e, 0x2e, 0xad, 0x6c, 0xca, 0x0e, 0xaf, 0x4f, 0x60, 0x23, 0xf2, 0x45, - 0x85, 0xc2, 0x6d, 0xf5, 0x7d, 0xb9, 0xd8, 0xd8, 0xd5, 0xa4, 0x55, 0x66, 0xd1, 0x27, 0x15, 0xf9, - 0x04, 0x2a, 0xb4, 0xeb, 0x06, 0xae, 0x31, 0x0d, 0x8e, 0xbd, 0xb0, 0xf6, 0xeb, 0xb2, 0x02, 0xd1, - 0x17, 0x50, 0xad, 0x8c, 0x44, 0x51, 0x0a, 0x77, 0x93, 0xf9, 0x3a, 0x1d, 0x85, 0x56, 0xed, 0x07, - 0x7c, 0x37, 0x89, 0x81, 0x8d, 0xd0, 0x62, 0x4f, 0x00, 0x8c, 0xe9, 0xd4, 0x39, 0xe7, 0x53, 0xf3, - 0x87, 0x34, 0x35, 0xb7, 0xa4, 0xa9, 0x59, 0x47, 0x24, 0xcd, 0xcd, 0xa2, 0x11, 0x7d, 0xb2, 0xc7, - 0x50, 0x9e, 0x7a, 0x41, 0xa8, 0x9b, 0x13, 0x87, 0xda, 0x5f, 0x97, 0xd7, 0xf6, 0x81, 0x17, 0x84, - 0xcd, 0x89, 0x43, 0x7b, 0xca, 0x34, 0xfe, 0x66, 0x1d, 0xb8, 0x94, 0x90, 0xdb, 0x06, 0x1d, 0xcb, - 0xd6, 0x76, 0xa8, 0xc4, 0x1b, 0x52, 0x89, 0x92, 0xfc, 0x16, 0x31, 0x49, 0x9b, 0xde, 0x22, 0x08, - 0x6d, 0x22, 0x3e, 0x06, 0x71, 0x60, 0x5e, 0x83, 0xab, 0x12, 0x04, 0x8d, 0x22, 0xf3, 0x9e, 0xc2, - 0xc6, 0x9c, 0x0a, 0x1b, 0x18, 0xd4, 0x9a, 0xf2, 0x4c, 0x96, 0xc2, 0x67, 0x2b, 0x11, 0x23, 0xc2, - 0x02, 0xea, 0x3b, 0xcf, 0x71, 0x66, 0x53, 0x21, 0x4a, 0x6b, 0x2d, 0xd1, 0x77, 0x04, 0xe4, 0x52, - 0x52, 0xfd, 0xa7, 0x59, 0x28, 0x44, 0xf6, 0x02, 0x2b, 0x41, 0xfe, 0xb0, 0xfb, 0xbc, 0xdb, 0x7b, - 0xd9, 0xe5, 0x37, 0x7d, 0xea, 0xfd, 0x7e, 0x4b, 0x1b, 0x28, 0x26, 0xab, 0x02, 0x50, 0x2c, 0xbf, - 0xde, 0x6f, 0xd4, 0xbb, 0xfc, 0xe6, 0x0f, 0xdd, 0x20, 0xe0, 0xe9, 0x75, 0xb6, 0x09, 0x95, 0xdd, - 0xc3, 0x2e, 0xc5, 0x47, 0x71, 0x50, 0x1a, 0x41, 0xad, 0xcf, 0xf8, 0xa9, 0x10, 0x07, 0x65, 0x10, - 0xb4, 0x5f, 0x1f, 0xb4, 0xb4, 0x76, 0x04, 0xca, 0x52, 0xa8, 0x55, 0xef, 0x50, 0x6b, 0x88, 0x9c, - 0x72, 0xec, 0x32, 0x6c, 0xc6, 0x6c, 0x51, 0x96, 0x4a, 0x1e, 0x6b, 0x76, 0xa0, 0xf5, 0x7e, 0xd4, - 0x6a, 0x0c, 0x14, 0xa0, 0x23, 0xa6, 0x67, 0xcf, 0x94, 0x12, 0x2b, 0x43, 0xa1, 0xd9, 0xee, 0x0f, - 0xda, 0xdd, 0xc6, 0x40, 0x29, 0x63, 0x85, 0x77, 0xdb, 0x9d, 0x41, 0x4b, 0x53, 0x2a, 0xac, 0x00, - 0x99, 0x1f, 0xf5, 0xda, 0x5d, 0xa5, 0x4a, 0x77, 0x1a, 0xea, 0xfb, 0x07, 0x9d, 0x96, 0xb2, 0x81, - 0xd0, 0x7e, 0x4f, 0x1b, 0x28, 0x0a, 0x42, 0x5f, 0xb6, 0xbb, 0xcd, 0xde, 0x4b, 0x65, 0x93, 0x15, - 0x21, 0x7b, 0xd8, 0xc5, 0x62, 0x18, 0xab, 0x40, 0x91, 0x3e, 0xf5, 0x7a, 0xa7, 0xa3, 0x5c, 0x92, - 0xce, 0xa5, 0xb6, 0x10, 0x45, 0xa7, 0x5c, 0x7d, 0xac, 0xc3, 0x65, 0x6c, 0x4b, 0x9c, 0x24, 0xea, - 0x2b, 0x98, 0xcf, 0x7e, 0xbb, 0x7b, 0xd8, 0x57, 0xae, 0x22, 0x31, 0x7d, 0x12, 0xa6, 0x86, 0xf9, - 0xb4, 0xbb, 0xd4, 0x95, 0x37, 0xf1, 0xbb, 0xd9, 0xea, 0xb4, 0x06, 0x2d, 0xe5, 0x16, 0xb6, 0x4a, - 0x6b, 0x1d, 0x74, 0xea, 0x8d, 0x96, 0xb2, 0x8d, 0x89, 0x4e, 0xaf, 0xf1, 0x5c, 0xef, 0x1d, 0x28, - 0xb7, 0xd9, 0x16, 0x28, 0xbd, 0xae, 0xde, 0x3c, 0x3c, 0xe8, 0xb4, 0x1b, 0xf5, 0x41, 0x4b, 0x7f, - 0xde, 0xfa, 0x5c, 0x51, 0xb1, 0xdb, 0x0f, 0xb4, 0x96, 0x2e, 0xf2, 0xba, 0xc3, 0x14, 0x28, 0xef, - 0x1e, 0xfe, 0xe4, 0x27, 0x9f, 0xeb, 0xa2, 0xdd, 0x6f, 0x61, 0xb5, 0xe6, 0x14, 0xfa, 0xe1, 0x73, - 0xe5, 0xde, 0x02, 0xa8, 0xff, 0x5c, 0x79, 0x1b, 0xfb, 0x2d, 0x1a, 0x08, 0xe5, 0x3e, 0x12, 0x68, - 0xad, 0xc6, 0xa1, 0xd6, 0x6f, 0xbf, 0x68, 0xe9, 0x8d, 0x41, 0x4b, 0xf9, 0x0e, 0x75, 0x54, 0xbb, - 0xfb, 0x5c, 0x79, 0x80, 0x2d, 0xc1, 0x2f, 0x3e, 0x3c, 0xef, 0x30, 0x06, 0xd5, 0x39, 0x2d, 0xc1, - 0xde, 0x45, 0x92, 0x1d, 0xad, 0x57, 0x6f, 0x36, 0xea, 0xfd, 0x81, 0xf2, 0x1e, 0x76, 0x43, 0xff, - 0xa0, 0xd3, 0x1e, 0x28, 0x0f, 0xb1, 0xad, 0xcf, 0xea, 0x83, 0xbd, 0x96, 0xa6, 0x3c, 0xc2, 0x91, - 0x1e, 0xb4, 0xf7, 0x5b, 0xba, 0xe8, 0xf6, 0xc7, 0x58, 0xc6, 0x6e, 0xbb, 0xd3, 0x51, 0x9e, 0xd0, - 0xd1, 0x4b, 0x5d, 0x1b, 0xb4, 0x69, 0xac, 0x3f, 0xc4, 0x0c, 0xea, 0x07, 0x07, 0x9d, 0xcf, 0x95, - 0x8f, 0xb0, 0x81, 0xfb, 0x87, 0x9d, 0x41, 0x5b, 0x3f, 0x3c, 0x68, 0xd6, 0x07, 0x2d, 0xe5, 0x63, - 0x9a, 0x08, 0xbd, 0xfe, 0xa0, 0xb9, 0xdf, 0x51, 0x3e, 0xa1, 0x3c, 0x69, 0x1a, 0x36, 0x3a, 0xbd, - 0x6e, 0x4b, 0x79, 0xaa, 0xfe, 0x16, 0x14, 0x22, 0x1b, 0x12, 0xb3, 0x69, 0x77, 0xbb, 0x2d, 0x4d, - 0x59, 0xc3, 0xa2, 0x3a, 0xad, 0xdd, 0x81, 0x92, 0xa2, 0x73, 0xa8, 0xf6, 0xb3, 0xbd, 0x81, 0xb2, - 0x8e, 0x9f, 0xbd, 0x43, 0xec, 0xb5, 0x34, 0x35, 0xb7, 0xb5, 0xdf, 0x56, 0x32, 0xf8, 0x55, 0xef, - 0x0e, 0xda, 0x4a, 0x96, 0xe6, 0x4d, 0xbb, 0xfb, 0xac, 0xd3, 0x52, 0x72, 0x08, 0xdd, 0xaf, 0x6b, - 0xcf, 0x95, 0x3c, 0xcf, 0xb4, 0xd9, 0xfa, 0x4c, 0x29, 0xb0, 0x1c, 0xac, 0x77, 0x1e, 0x2b, 0x45, - 0x04, 0x35, 0x5b, 0xcd, 0xc3, 0x03, 0x05, 0xd4, 0xfb, 0x90, 0xaf, 0x1f, 0x1d, 0xed, 0xa3, 0x89, - 0x8e, 0xad, 0x3b, 0xec, 0x74, 0xf8, 0x3a, 0xda, 0xe9, 0x0d, 0x06, 0xbd, 0x7d, 0x25, 0x85, 0x33, - 0x77, 0xd0, 0x3b, 0x50, 0xd6, 0xd5, 0x36, 0x14, 0xa2, 0x0d, 0x53, 0xba, 0xa7, 0x53, 0x80, 0xcc, - 0x81, 0xd6, 0x7a, 0xc1, 0x0f, 0x4f, 0xbb, 0xad, 0xcf, 0xb0, 0x9a, 0xf8, 0x85, 0x19, 0xa5, 0xb1, - 0x20, 0x7e, 0xa1, 0x86, 0x2e, 0xea, 0x74, 0xda, 0xdd, 0x56, 0x5d, 0x53, 0xb2, 0xea, 0x47, 0x89, - 0x73, 0x29, 0x21, 0x5b, 0xb0, 0xf8, 0x7a, 0x5b, 0x14, 0xdf, 0x7e, 0xd6, 0xed, 0x69, 0x2d, 0x7e, - 0xf3, 0x47, 0x74, 0xe4, 0xba, 0xfa, 0x0e, 0x14, 0x63, 0xb9, 0x88, 0x13, 0xab, 0xa1, 0xf5, 0xfa, - 0x7d, 0xde, 0xef, 0x6b, 0x98, 0xa6, 0xbe, 0xe1, 0xe9, 0x94, 0xda, 0x87, 0xcd, 0x48, 0x24, 0x53, - 0x50, 0x34, 0xd9, 0x16, 0x5b, 0x90, 0xed, 0x58, 0xaf, 0x2c, 0x27, 0x8a, 0xee, 0xa5, 0x04, 0x42, - 0x7b, 0xc3, 0x2f, 0xda, 0xf1, 0xe5, 0x4c, 0x4a, 0xa0, 0x0e, 0xd6, 0x95, 0xee, 0x87, 0x52, 0x54, - 0xf9, 0xef, 0xa5, 0xa0, 0x10, 0x0b, 0xfa, 0xbb, 0xb0, 0x3e, 0xe8, 0x0b, 0xbf, 0xf6, 0xd6, 0xc3, - 0xf9, 0x75, 0xf8, 0x41, 0xf4, 0xa5, 0xad, 0x0f, 0xfa, 0xec, 0x5d, 0xc8, 0xf1, 0xeb, 0x6c, 0xe2, - 0x40, 0x62, 0x2b, 0xb9, 0x79, 0x0c, 0x08, 0xa7, 0x09, 0x1a, 0xf6, 0x11, 0x14, 0xe3, 0xda, 0x0a, - 0xff, 0xc6, 0xd5, 0x24, 0x43, 0x8c, 0xd6, 0xe6, 0x94, 0x6a, 0x07, 0xaa, 0xc9, 0x0c, 0xd9, 0x4d, - 0x00, 0x9e, 0xa5, 0xe4, 0xd7, 0x91, 0x20, 0xec, 0x3a, 0x44, 0xb7, 0xec, 0x9a, 0x54, 0xb1, 0x4a, - 0x7c, 0xeb, 0xae, 0xa9, 0xfe, 0xb5, 0x34, 0xc0, 0x5c, 0x55, 0xc4, 0x8e, 0x88, 0xbd, 0x36, 0x59, - 0x71, 0xa4, 0xf9, 0x06, 0x14, 0x1d, 0xcf, 0x30, 0xe5, 0xdb, 0xf0, 0x05, 0x04, 0xd0, 0xd0, 0xc8, - 0x37, 0x4e, 0x8a, 0x3c, 0x9e, 0x80, 0x5d, 0x81, 0xdc, 0xd8, 0xf3, 0x27, 0x46, 0x28, 0x42, 0xb9, - 0x45, 0x0a, 0x25, 0x3e, 0x3f, 0x66, 0x43, 0x85, 0xd9, 0xa5, 0x68, 0x6e, 0x1c, 0x83, 0xb2, 0x00, - 0x76, 0x10, 0x86, 0x86, 0x91, 0xe5, 0x8e, 0x1c, 0x2f, 0xb0, 0x4c, 0x7d, 0xc8, 0x43, 0x34, 0xca, - 0x1a, 0x44, 0xa0, 0x9d, 0x73, 0xde, 0x5a, 0x7f, 0x62, 0xbb, 0x46, 0x28, 0x9c, 0xcf, 0xd4, 0xda, - 0x08, 0x82, 0xd5, 0xfd, 0x22, 0xf0, 0x84, 0x13, 0x87, 0x9f, 0xd8, 0x15, 0x10, 0x40, 0xd5, 0x7d, - 0x13, 0xc0, 0x0a, 0x46, 0xc6, 0x94, 0x67, 0x5e, 0xa4, 0xcc, 0x8b, 0x02, 0xb2, 0x73, 0xce, 0x3a, - 0x50, 0x1d, 0x0c, 0x71, 0x87, 0xf2, 0xd0, 0x1e, 0x6f, 0x78, 0x8e, 0x70, 0xaf, 0xdc, 0x5d, 0xd4, - 0xa9, 0x1f, 0x26, 0xc9, 0xf8, 0xd1, 0xe2, 0x02, 0xef, 0xf5, 0x3a, 0x5c, 0x5a, 0x41, 0xf6, 0x8d, - 0x62, 0xac, 0x9c, 0x68, 0x74, 0xea, 0x61, 0x48, 0xb7, 0x27, 0xe2, 0xcd, 0x38, 0x15, 0xc5, 0x80, - 0xf3, 0x7d, 0xf8, 0x0d, 0x8a, 0xa2, 0x10, 0xe1, 0x79, 0x62, 0x90, 0xe2, 0xb0, 0xbb, 0x7b, 0xb0, - 0x81, 0xc8, 0xb1, 0x6d, 0x39, 0xa6, 0x20, 0xe1, 0xc1, 0xff, 0x95, 0x91, 0xe7, 0xec, 0x22, 0x94, - 0xe8, 0xd4, 0xbf, 0x92, 0x05, 0x98, 0x9b, 0x61, 0x89, 0xd3, 0xcd, 0x54, 0xf2, 0x74, 0xf3, 0x31, - 0x5c, 0x11, 0x57, 0x43, 0xe2, 0x23, 0x42, 0xdb, 0xd5, 0x87, 0x46, 0x74, 0x90, 0xcc, 0x04, 0x96, - 0x9f, 0x12, 0xb6, 0xdd, 0x1d, 0x03, 0x95, 0xba, 0x0d, 0x99, 0x27, 0x3c, 0x9f, 0x26, 0x0f, 0xc2, - 0x65, 0x55, 0x61, 0xce, 0x3e, 0x38, 0x9f, 0xb2, 0xf7, 0xe1, 0xb2, 0x6f, 0x8d, 0x7d, 0x2b, 0x38, - 0xd6, 0xc3, 0x40, 0x2e, 0x8c, 0xc7, 0x6b, 0x6d, 0x0a, 0xe4, 0x20, 0x88, 0xcb, 0x7a, 0x1f, 0x2e, - 0x0b, 0x03, 0x6d, 0xa1, 0x7a, 0xfc, 0x0e, 0xf1, 0x26, 0x47, 0xca, 0xb5, 0x7b, 0x13, 0x40, 0xd8, - 0xa6, 0xd1, 0x9b, 0x12, 0x05, 0xad, 0xc8, 0xed, 0x50, 0x71, 0xd3, 0x92, 0x0c, 0x4c, 0x71, 0xf4, - 0xc3, 0x13, 0x4c, 0x85, 0x0c, 0x8a, 0x53, 0x3a, 0xa6, 0xa8, 0x3e, 0xae, 0x3e, 0xa4, 0x37, 0x33, - 0xe8, 0xe6, 0xab, 0x67, 0x5a, 0x1a, 0xe1, 0xd8, 0x7b, 0x70, 0x49, 0x6e, 0x76, 0x74, 0xed, 0xbb, - 0x44, 0x15, 0x51, 0xe6, 0x0d, 0xd5, 0xf8, 0x05, 0xf0, 0x77, 0x80, 0x49, 0x35, 0x8f, 0xa8, 0xcb, - 0x44, 0xbd, 0x11, 0x57, 0x5b, 0x10, 0xbf, 0x0d, 0x54, 0x45, 0xee, 0x15, 0xae, 0x2c, 0x5b, 0x63, - 0x88, 0x24, 0x0f, 0xf2, 0xfb, 0x70, 0x79, 0xde, 0x3a, 0xdd, 0x08, 0xf5, 0xf0, 0xd8, 0xd2, 0x2d, - 0xd7, 0xa4, 0xfb, 0x3c, 0x05, 0x6d, 0x33, 0x6e, 0x68, 0x3d, 0x1c, 0x1c, 0x5b, 0x68, 0x4f, 0x49, - 0xfe, 0xb1, 0x8d, 0xaf, 0xf6, 0x8f, 0x7d, 0x0c, 0xb5, 0xc4, 0x91, 0xa7, 0xdc, 0xdd, 0xfc, 0x3e, - 0xdc, 0x96, 0x7c, 0xd0, 0x19, 0xf7, 0xf8, 0x03, 0xd8, 0x3c, 0x36, 0x02, 0x3d, 0xc1, 0x4b, 0x6e, - 0xbb, 0x82, 0xb6, 0x71, 0x6c, 0x04, 0x07, 0x12, 0x8f, 0xfa, 0x07, 0x29, 0xa8, 0x26, 0x0d, 0x53, - 0x7e, 0x1f, 0xc2, 0x99, 0x4d, 0x5c, 0x1e, 0xdd, 0x90, 0xd5, 0xa2, 0x24, 0xae, 0x85, 0xe9, 0x89, - 0xce, 0x53, 0xd1, 0x5a, 0x98, 0x9e, 0x34, 0x28, 0xcd, 0xbe, 0x03, 0xf9, 0xe9, 0x09, 0x17, 0x0e, - 0x17, 0xcd, 0xbe, 0xdc, 0x94, 0x87, 0x95, 0x7e, 0x07, 0xf2, 0x33, 0x41, 0x9a, 0xb9, 0x88, 0x74, - 0x46, 0xa4, 0xea, 0x3f, 0x5b, 0x87, 0xb2, 0xec, 0x92, 0xf9, 0x3a, 0x27, 0xa1, 0xdf, 0xe8, 0x04, - 0x7b, 0x9b, 0xa2, 0xcc, 0x74, 0x8a, 0x61, 0xc5, 0x7e, 0xe2, 0xc7, 0xa0, 0x70, 0x6c, 0x04, 0xf5, - 0x59, 0xe8, 0x35, 0x3c, 0x7e, 0xfa, 0xe2, 0x39, 0x51, 0x6c, 0x2b, 0x5f, 0x19, 0x28, 0x13, 0x44, - 0x58, 0xeb, 0xfb, 0x22, 0x74, 0x9e, 0x2e, 0xcb, 0x50, 0xf4, 0x44, 0x76, 0x69, 0xbe, 0x94, 0xa3, - 0xbb, 0x32, 0x14, 0x18, 0xf1, 0x18, 0x36, 0xe6, 0x81, 0xca, 0x51, 0xc0, 0xc5, 0x22, 0x4b, 0x25, - 0x8e, 0x52, 0x16, 0xb7, 0x63, 0x2b, 0x76, 0xa0, 0x7b, 0x8e, 0x19, 0xdd, 0x88, 0xc8, 0x47, 0xae, - 0xee, 0x9e, 0x63, 0x8a, 0xfb, 0x52, 0x9c, 0xc6, 0xb5, 0x4e, 0x23, 0x9a, 0xd8, 0x1d, 0xde, 0xb5, - 0x4e, 0xc5, 0xcd, 0x88, 0x3f, 0x4e, 0xc1, 0xe6, 0x92, 0x0b, 0x05, 0x25, 0xe7, 0xfc, 0xad, 0x16, - 0xfc, 0x64, 0xb7, 0xa1, 0x3c, 0x31, 0xc2, 0xd1, 0xb1, 0x3e, 0xf5, 0xad, 0xb1, 0x7d, 0x16, 0x3d, - 0x38, 0x43, 0xb0, 0x03, 0x02, 0x51, 0x10, 0xca, 0x74, 0x4a, 0x8e, 0xa3, 0x89, 0x1d, 0x0a, 0xc1, - 0x07, 0x04, 0xea, 0x90, 0x6b, 0x39, 0x0a, 0x50, 0xcb, 0x5c, 0x10, 0xa0, 0x76, 0x1d, 0x8a, 0xae, - 0x17, 0xea, 0x9e, 0xab, 0x4f, 0x4f, 0xc4, 0x75, 0xea, 0xbc, 0xeb, 0x85, 0x3d, 0xf7, 0xe0, 0x44, - 0xbd, 0x01, 0xb9, 0x76, 0xec, 0xc6, 0x89, 0x23, 0x2c, 0xd2, 0xe2, 0xc5, 0x05, 0x0f, 0x8a, 0x0d, - 0x7a, 0xbd, 0x61, 0xdf, 0x98, 0xb2, 0x07, 0x90, 0x9e, 0x18, 0x53, 0x11, 0x81, 0x51, 0x8b, 0xcf, - 0x52, 0x38, 0xf6, 0xe1, 0xbe, 0x31, 0xe5, 0x9b, 0x08, 0x12, 0x5d, 0xff, 0x18, 0x0a, 0x11, 0xe0, - 0x1b, 0x6d, 0x17, 0xff, 0x72, 0x1d, 0x8a, 0x4d, 0xd9, 0x6d, 0x8b, 0x16, 0x6d, 0xe8, 0xcf, 0x5c, - 0xd4, 0xbd, 0xa2, 0x7b, 0xe8, 0x23, 0xc3, 0x1d, 0x08, 0x50, 0x34, 0x4d, 0xd7, 0xbf, 0x62, 0x9a, - 0xde, 0x00, 0xf0, 0xc9, 0x8b, 0x41, 0x8e, 0x8c, 0x74, 0x1c, 0xed, 0xd7, 0x36, 0xdb, 0xe6, 0xd9, - 0xea, 0xe3, 0xfc, 0xcc, 0xd7, 0x3f, 0xce, 0xcf, 0xae, 0x3c, 0xce, 0xbf, 0x37, 0xdf, 0x2a, 0x70, - 0xba, 0x62, 0xc1, 0x45, 0xbe, 0x61, 0x4d, 0xe3, 0xa0, 0x7f, 0x2c, 0xfd, 0xbb, 0x50, 0x8d, 0x5a, - 0x27, 0xf2, 0x83, 0xc4, 0x3d, 0x03, 0x81, 0xe3, 0x8e, 0xe0, 0x4a, 0x28, 0x27, 0x93, 0xcb, 0xaf, - 0xf4, 0x9a, 0x38, 0x87, 0x3f, 0x4c, 0x01, 0x13, 0x56, 0xf7, 0xee, 0xcc, 0x71, 0x06, 0xd6, 0x19, - 0xad, 0xf2, 0x07, 0xb0, 0x29, 0xbc, 0xb8, 0x52, 0x94, 0x90, 0x38, 0x58, 0xe3, 0x88, 0xf9, 0xc1, - 0xda, 0xaa, 0xab, 0x60, 0xeb, 0x2b, 0xaf, 0x82, 0xad, 0xbe, 0x62, 0x76, 0x0b, 0x4a, 0xf2, 0x45, - 0x2a, 0xae, 0x5a, 0x81, 0x31, 0xbf, 0x43, 0xf5, 0xaf, 0xd7, 0x01, 0xe6, 0x9e, 0x81, 0x5f, 0x75, - 0x2c, 0xc6, 0x8a, 0x21, 0x49, 0xaf, 0x1a, 0x92, 0xfb, 0xa0, 0xc8, 0x74, 0xd2, 0x8d, 0xbe, 0xea, - 0x9c, 0x30, 0x52, 0x59, 0xec, 0x40, 0xbe, 0x75, 0x45, 0x81, 0x57, 0xe2, 0xbc, 0x59, 0x44, 0x65, - 0x91, 0x3c, 0x15, 0xbb, 0x71, 0xc1, 0x0e, 0xb8, 0x7c, 0x65, 0x9f, 0xc2, 0xb5, 0x98, 0x53, 0x3f, - 0xb5, 0xc3, 0x63, 0x6f, 0x16, 0x0a, 0x2f, 0x42, 0x20, 0x24, 0xce, 0x95, 0x28, 0xa7, 0x97, 0x1c, - 0xcd, 0xa5, 0x48, 0x80, 0x4a, 0xf7, 0x78, 0xe6, 0x38, 0x7a, 0x68, 0x9d, 0x85, 0xe2, 0x6a, 0x7b, - 0x2d, 0xe1, 0x54, 0x91, 0x86, 0x57, 0x2b, 0x8c, 0x45, 0x42, 0xfd, 0xfb, 0x69, 0xc8, 0xfe, 0x78, - 0x66, 0xf9, 0xe7, 0xec, 0x63, 0x28, 0x06, 0xe1, 0x24, 0x94, 0x0f, 0x37, 0xaf, 0xf1, 0x0c, 0x08, - 0x4f, 0x67, 0x93, 0xd6, 0xc4, 0x72, 0x43, 0xee, 0x6d, 0x44, 0x5a, 0xda, 0x4c, 0xb6, 0x20, 0x1b, - 0x84, 0xd6, 0x34, 0x10, 0x51, 0x54, 0x3c, 0xc1, 0xb6, 0x21, 0xeb, 0x7a, 0xa6, 0x15, 0x24, 0x63, - 0xa5, 0xba, 0xa8, 0x3d, 0x70, 0x04, 0x53, 0x21, 0x17, 0x8f, 0xf8, 0xd2, 0x01, 0x23, 0xc7, 0x50, - 0xf4, 0xbb, 0x65, 0x98, 0xb6, 0x7b, 0x14, 0xdd, 0x90, 0x8c, 0xd3, 0xb8, 0x4d, 0x92, 0xb2, 0x6e, - 0x1c, 0x45, 0xd7, 0x95, 0x45, 0x92, 0x6d, 0x43, 0x09, 0x3f, 0x5f, 0xfa, 0x76, 0x68, 0xf5, 0x9f, - 0x44, 0x92, 0x5a, 0x02, 0xa1, 0xaa, 0x6d, 0x5a, 0xa1, 0x35, 0x0a, 0xfb, 0x5f, 0x8a, 0x00, 0x28, - 0x8a, 0x93, 0x89, 0x20, 0xec, 0xbb, 0xc0, 0x86, 0xc6, 0xe8, 0xe4, 0xc8, 0xf7, 0x66, 0xae, 0xa9, - 0x7f, 0x39, 0xb3, 0x7c, 0xdb, 0x8a, 0x02, 0x9e, 0x4a, 0x52, 0xa7, 0x68, 0x9b, 0x73, 0xb2, 0x1f, - 0x73, 0x2a, 0xd5, 0x84, 0x4a, 0xa2, 0xab, 0x96, 0xbc, 0x3b, 0xfd, 0x56, 0xa7, 0xd5, 0x18, 0x70, - 0xb3, 0x50, 0xb8, 0x14, 0xd6, 0x65, 0x97, 0x44, 0x5a, 0xf2, 0x55, 0x64, 0x24, 0xdb, 0x31, 0x4b, - 0x9e, 0x8e, 0x96, 0xf6, 0xac, 0xa5, 0xe4, 0xd4, 0x3f, 0x5a, 0x87, 0xcd, 0x81, 0x6f, 0xb8, 0x81, - 0xc1, 0x35, 0x09, 0x37, 0xf4, 0x3d, 0x87, 0x7d, 0x17, 0x0a, 0xe1, 0xc8, 0x91, 0x87, 0xf0, 0x56, - 0x24, 0x30, 0x16, 0x48, 0x1f, 0x0e, 0x46, 0xdc, 0x6d, 0x9c, 0x0f, 0xf9, 0x07, 0x7b, 0x0f, 0xb2, - 0x43, 0xeb, 0xc8, 0x76, 0x85, 0xcc, 0xbc, 0xbc, 0xc8, 0xb8, 0x83, 0xc8, 0xbd, 0x35, 0x8d, 0x53, - 0xb1, 0xf7, 0x21, 0x37, 0xf2, 0x26, 0xd1, 0xc6, 0x33, 0xbf, 0x69, 0x23, 0x15, 0x84, 0xd8, 0xbd, - 0x35, 0x4d, 0xd0, 0xb1, 0x8f, 0xa1, 0xe0, 0x7b, 0x8e, 0x83, 0x3d, 0x26, 0xb6, 0xa4, 0xda, 0x22, - 0x8f, 0x26, 0xf0, 0x7b, 0x6b, 0x5a, 0x4c, 0xab, 0x3e, 0x84, 0xbc, 0xa8, 0x2c, 0x76, 0xc0, 0x4e, - 0xeb, 0x59, 0x5b, 0x74, 0x64, 0xa3, 0xb7, 0xbf, 0xdf, 0x1e, 0xf0, 0x6b, 0x84, 0x5a, 0xaf, 0xd3, - 0xd9, 0xa9, 0x37, 0x9e, 0x2b, 0xeb, 0x3b, 0x05, 0xc8, 0x71, 0x07, 0xa1, 0xfa, 0xdb, 0x29, 0xd8, - 0x58, 0x68, 0x00, 0x7b, 0x0a, 0x99, 0x09, 0x6a, 0xb6, 0xbc, 0x7b, 0xee, 0xae, 0x6c, 0xa5, 0x94, - 0xe6, 0xfa, 0x2e, 0x72, 0xa8, 0x9f, 0x42, 0x35, 0x09, 0x97, 0x3c, 0x08, 0x15, 0x28, 0x6a, 0xad, - 0x7a, 0x53, 0xef, 0x75, 0xd1, 0x6e, 0x47, 0x3b, 0x9e, 0x92, 0x2f, 0xb5, 0x36, 0x19, 0xfd, 0xbf, - 0x01, 0xca, 0x62, 0xc7, 0xb0, 0x67, 0x68, 0xbb, 0x4c, 0xa6, 0x8e, 0x45, 0x2a, 0xa2, 0x34, 0x64, - 0x37, 0x57, 0xf4, 0xa4, 0x20, 0xa3, 0x11, 0xab, 0x8e, 0x12, 0x69, 0xf5, 0x37, 0x81, 0x2d, 0xf7, - 0xe0, 0xaf, 0x2e, 0xfb, 0xff, 0x9e, 0x82, 0xcc, 0x81, 0x63, 0xb8, 0xec, 0x0e, 0x64, 0xe9, 0xf9, - 0x0b, 0x21, 0x79, 0xe5, 0x75, 0x80, 0xd3, 0x82, 0x70, 0xec, 0x1d, 0x48, 0x87, 0xa3, 0xe8, 0xf6, - 0xe2, 0xd5, 0x0b, 0x26, 0xdf, 0xde, 0x9a, 0x86, 0x54, 0xec, 0x3e, 0xa4, 0x4d, 0x33, 0x8a, 0x1a, - 0x16, 0x3e, 0x05, 0xb4, 0x28, 0x9b, 0xd6, 0xd8, 0x76, 0x6d, 0xf1, 0x5c, 0x07, 0x92, 0xb0, 0xb7, - 0x20, 0x6d, 0x8e, 0x9c, 0x64, 0x08, 0x38, 0xb7, 0x3d, 0xe3, 0x0c, 0xcd, 0x91, 0x83, 0x1a, 0x58, - 0xe8, 0x9f, 0xeb, 0xfe, 0xcc, 0xa5, 0x30, 0xa8, 0x40, 0x58, 0x45, 0x25, 0xd4, 0x3f, 0x66, 0x14, - 0x4b, 0x15, 0x88, 0x6b, 0x50, 0x53, 0xdf, 0x9a, 0x1a, 0x7e, 0x6c, 0x0f, 0xd9, 0xc1, 0x01, 0x07, - 0xec, 0xe4, 0x80, 0x5e, 0x15, 0x54, 0xdf, 0xa5, 0xb7, 0x19, 0x50, 0xb1, 0x56, 0xa3, 0xaf, 0x15, - 0x0f, 0x50, 0x09, 0x8c, 0xfa, 0x17, 0x69, 0x28, 0x49, 0xf5, 0x61, 0x1f, 0x42, 0xc1, 0x4c, 0x2e, - 0xc4, 0x6b, 0x4b, 0x95, 0x7e, 0xd8, 0x8c, 0x96, 0xa0, 0x29, 0xa6, 0x37, 0x9d, 0x49, 0x84, 0xfa, - 0x2b, 0xc3, 0xb7, 0xf9, 0x8b, 0x3c, 0xeb, 0xf2, 0xe1, 0x40, 0xdf, 0x0a, 0x5f, 0x44, 0x98, 0xbd, - 0x35, 0xad, 0x1c, 0x48, 0x69, 0xd2, 0xfe, 0x45, 0x93, 0xd2, 0x89, 0x97, 0x8d, 0x38, 0x70, 0x6f, - 0x4d, 0x8b, 0xf0, 0x48, 0x6a, 0x9d, 0x59, 0xa3, 0x59, 0x18, 0x69, 0xff, 0x95, 0xa8, 0x41, 0x04, - 0xa4, 0xe7, 0xd5, 0xf8, 0x27, 0x7b, 0x8c, 0x72, 0xd2, 0x70, 0x1c, 0x8f, 0xd4, 0xac, 0xac, 0xec, - 0xaa, 0x6f, 0xc6, 0x70, 0xfe, 0x9c, 0x5b, 0x94, 0x62, 0xf7, 0x20, 0xeb, 0x85, 0xc7, 0x56, 0xa4, - 0x53, 0x47, 0xaf, 0x3c, 0x20, 0xa8, 0xd9, 0xe8, 0xe0, 0x4c, 0x21, 0xb4, 0xfa, 0xf3, 0x14, 0xe4, - 0x45, 0x0f, 0xb0, 0x4d, 0xa8, 0xf4, 0x5b, 0x03, 0xfd, 0x45, 0x5d, 0x6b, 0xd7, 0x77, 0x3a, 0x2d, - 0x11, 0xb9, 0xfe, 0x4c, 0xab, 0x77, 0x85, 0x9c, 0xd4, 0x5a, 0x2f, 0x7a, 0xcf, 0x5b, 0xdc, 0x2d, - 0xd7, 0x6c, 0x75, 0x3f, 0x57, 0xd2, 0xdc, 0x35, 0xdd, 0x3a, 0xa8, 0x6b, 0x28, 0x25, 0x4b, 0x90, - 0x6f, 0x7d, 0xd6, 0x6a, 0x1c, 0x92, 0x98, 0xac, 0x02, 0x34, 0x5b, 0xf5, 0x4e, 0xa7, 0xd7, 0x40, - 0xb1, 0x99, 0x63, 0x0c, 0xaa, 0x0d, 0xad, 0x55, 0x1f, 0xb4, 0xf4, 0x7a, 0xa3, 0xd1, 0x3b, 0xec, - 0x0e, 0x94, 0x3c, 0x96, 0x58, 0xef, 0x0c, 0x5a, 0x5a, 0x0c, 0xa2, 0x97, 0x77, 0x9a, 0x5a, 0xef, - 0x20, 0x86, 0x14, 0x77, 0x8a, 0x68, 0x89, 0xd1, 0x58, 0xa9, 0xff, 0x70, 0x13, 0xaa, 0xc9, 0xa9, - 0xc9, 0x3e, 0x81, 0x82, 0x69, 0x26, 0xc6, 0xf8, 0xc6, 0xaa, 0x29, 0xfc, 0xb0, 0x69, 0x46, 0xc3, - 0xcc, 0x3f, 0xd8, 0xed, 0x68, 0x21, 0xad, 0x2f, 0x2d, 0xa4, 0x68, 0x19, 0xfd, 0x00, 0x36, 0xc4, - 0x2b, 0x09, 0xa6, 0x11, 0x1a, 0x43, 0x23, 0xb0, 0x92, 0xab, 0xa4, 0x41, 0xc8, 0xa6, 0xc0, 0xed, - 0xad, 0x69, 0xd5, 0x51, 0x02, 0xc2, 0xbe, 0x07, 0x55, 0x83, 0x8c, 0xed, 0x98, 0x3f, 0x23, 0x2b, - 0x90, 0x75, 0xc4, 0x49, 0xec, 0x15, 0x43, 0x06, 0xe0, 0x44, 0x34, 0x7d, 0x6f, 0x3a, 0x67, 0xce, - 0x26, 0x4e, 0xa9, 0x7c, 0x6f, 0x2a, 0xf1, 0x96, 0x4d, 0x29, 0xcd, 0x3e, 0x86, 0xb2, 0xa8, 0xf9, - 0xdc, 0xe1, 0x10, 0x2f, 0x59, 0x5e, 0x6d, 0x52, 0x08, 0xf7, 0xd6, 0xb4, 0xd2, 0x68, 0x9e, 0x64, - 0x4f, 0x50, 0x0b, 0x9c, 0xab, 0xcf, 0x79, 0x79, 0xae, 0x51, 0x6d, 0x23, 0x2e, 0x30, 0xe2, 0x14, - 0x7b, 0x1f, 0x80, 0xea, 0xc9, 0x79, 0x0a, 0x89, 0x78, 0x0f, 0xdf, 0x9b, 0x46, 0x2c, 0x45, 0x33, - 0x4a, 0x48, 0xd5, 0xe3, 0xee, 0xa2, 0xe2, 0x72, 0xf5, 0xc8, 0x65, 0x34, 0xaf, 0x1e, 0xf7, 0x34, - 0xc5, 0xd5, 0xe3, 0x6c, 0xb0, 0x54, 0xbd, 0x88, 0x8b, 0x57, 0x8f, 0x33, 0x45, 0xd5, 0xe3, 0x3c, - 0xa5, 0xc5, 0xea, 0x45, 0x2c, 0x54, 0x3d, 0xce, 0xf1, 0xbd, 0x25, 0xbd, 0xbf, 0x7c, 0xa1, 0xde, - 0x8f, 0xc3, 0x96, 0xd4, 0xfc, 0xbf, 0x07, 0xd5, 0xe0, 0xd8, 0x3b, 0x95, 0x04, 0x48, 0x45, 0xe6, - 0xee, 0x1f, 0x7b, 0xa7, 0xb2, 0x04, 0xa9, 0x04, 0x32, 0x00, 0x6b, 0xcb, 0x9b, 0x48, 0xd7, 0xb3, - 0xab, 0x72, 0x6d, 0xa9, 0x85, 0x2f, 0x6c, 0xeb, 0x14, 0x6b, 0x6b, 0x44, 0x09, 0xec, 0x94, 0xb9, - 0xf3, 0x25, 0x10, 0xee, 0x94, 0x44, 0xd4, 0x83, 0x28, 0x09, 0x62, 0x37, 0x4c, 0x80, 0x73, 0x6b, - 0xe6, 0xca, 0x6c, 0x8a, 0x3c, 0xb7, 0x0e, 0xdd, 0x04, 0x63, 0x99, 0x93, 0x0a, 0xd6, 0xf9, 0xaa, - 0x08, 0xac, 0x2f, 0x67, 0x96, 0x3b, 0xb2, 0x44, 0x34, 0x54, 0x62, 0x55, 0xf4, 0x05, 0x6e, 0xbe, - 0x2a, 0x22, 0x48, 0x3c, 0xaf, 0x63, 0x76, 0xb6, 0x38, 0xaf, 0x25, 0x66, 0x9a, 0xd7, 0x31, 0x6b, - 0xbc, 0xa0, 0x62, 0xde, 0x4b, 0x4b, 0x0b, 0x4a, 0x62, 0xe6, 0x0b, 0x2a, 0xe6, 0x7e, 0x02, 0x62, - 0x36, 0xf1, 0xce, 0x4d, 0xc4, 0x4c, 0xf1, 0x5a, 0x8b, 0xde, 0x85, 0x51, 0x9c, 0xc2, 0xb9, 0xea, - 0x5b, 0x68, 0x67, 0x88, 0xa9, 0x70, 0x59, 0x9e, 0xab, 0x1a, 0x61, 0xe2, 0xa5, 0xe4, 0xcf, 0x93, - 0x52, 0x61, 0x53, 0x3b, 0xf4, 0x6b, 0xe6, 0x72, 0x61, 0x07, 0x76, 0xe8, 0xcf, 0x0b, 0xc3, 0x14, - 0x7b, 0x0f, 0x68, 0x1a, 0x72, 0x16, 0x4b, 0x16, 0xdd, 0xd8, 0x2d, 0x82, 0xa1, 0x60, 0x8a, 0x6f, - 0x9c, 0x2c, 0xa2, 0x8c, 0x91, 0x39, 0xaa, 0x8d, 0xe5, 0xc9, 0xc2, 0x8b, 0x68, 0x34, 0x1b, 0x38, - 0x59, 0x38, 0x51, 0xc3, 0x1c, 0xb1, 0x07, 0x40, 0xdc, 0x44, 0x7f, 0x94, 0x78, 0x45, 0xc8, 0xf7, - 0xa6, 0x9c, 0x3a, 0x8f, 0x04, 0x48, 0x8b, 0x2d, 0x70, 0x3c, 0x37, 0x6a, 0xf8, 0x71, 0xa2, 0x05, - 0x88, 0x88, 0x85, 0xc1, 0x28, 0x4e, 0xa9, 0xff, 0x4f, 0x0e, 0xf2, 0x42, 0xd6, 0xb2, 0x4b, 0xb0, - 0x21, 0x44, 0x7e, 0xb3, 0x3e, 0xa8, 0xef, 0xd4, 0xfb, 0xa8, 0xa4, 0x31, 0xa8, 0x72, 0x99, 0x1f, - 0xc3, 0x52, 0xb8, 0x0f, 0x90, 0xd0, 0x8f, 0x41, 0xeb, 0xb8, 0x0f, 0x08, 0x5e, 0xfe, 0x5a, 0x5b, - 0x9a, 0x6d, 0x40, 0x89, 0x33, 0x72, 0x00, 0x5d, 0xa4, 0x23, 0x2e, 0x9e, 0xce, 0x4a, 0x2c, 0xfc, - 0x94, 0x2a, 0x37, 0x67, 0xe1, 0x80, 0x7c, 0xcc, 0x12, 0x1d, 0x63, 0x31, 0xa8, 0x0e, 0xb4, 0xc3, - 0x6e, 0x63, 0x5e, 0x4e, 0x91, 0x2e, 0x3f, 0xf1, 0x6c, 0x5e, 0xb4, 0x5b, 0x2f, 0x15, 0x40, 0x26, - 0x9e, 0x0b, 0xa5, 0x4b, 0xa8, 0x66, 0x52, 0x26, 0x94, 0x2c, 0xb3, 0xab, 0x70, 0xa9, 0xbf, 0xd7, - 0x7b, 0xa9, 0x73, 0xa6, 0xb8, 0x09, 0x15, 0xb6, 0x05, 0x8a, 0x84, 0xe0, 0xd9, 0x57, 0xb1, 0x48, - 0x82, 0x46, 0x84, 0x7d, 0x65, 0x83, 0x0e, 0x82, 0x11, 0x36, 0xe0, 0xfb, 0xae, 0x82, 0x4d, 0xe1, - 0xac, 0xbd, 0xce, 0xe1, 0x7e, 0xb7, 0xaf, 0x6c, 0x62, 0x25, 0x08, 0xc2, 0x6b, 0xce, 0xe2, 0x6c, - 0xe6, 0xbb, 0xf5, 0x25, 0xda, 0xc0, 0x11, 0xf6, 0xb2, 0xae, 0x75, 0xdb, 0xdd, 0x67, 0x7d, 0x65, - 0x2b, 0xce, 0xb9, 0xa5, 0x69, 0x3d, 0xad, 0xaf, 0x5c, 0x8e, 0x01, 0xfd, 0x41, 0x7d, 0x70, 0xd8, - 0x57, 0xae, 0xc4, 0xb5, 0x3c, 0xd0, 0x7a, 0x8d, 0x56, 0xbf, 0xdf, 0x69, 0xf7, 0x07, 0xca, 0x55, - 0x76, 0x19, 0x36, 0xe7, 0x35, 0x8a, 0x88, 0x6b, 0x52, 0x45, 0xb5, 0x67, 0xad, 0x81, 0x72, 0x2d, - 0xae, 0x46, 0xa3, 0xd7, 0xe9, 0xd4, 0xe9, 0x08, 0xf3, 0x3a, 0x12, 0xd1, 0x59, 0xae, 0x68, 0xcd, - 0x1b, 0x58, 0xaf, 0xc3, 0xae, 0x0c, 0xba, 0x21, 0x4d, 0x8d, 0x7e, 0xeb, 0xc7, 0x87, 0xad, 0x6e, - 0xa3, 0xa5, 0xbc, 0x39, 0x9f, 0x1a, 0x31, 0xec, 0x66, 0x3c, 0x35, 0x62, 0xd0, 0xad, 0xb8, 0xcc, - 0x08, 0xd4, 0x57, 0xb6, 0x31, 0x3f, 0x51, 0x8f, 0x6e, 0xb7, 0xd5, 0x18, 0x60, 0x5b, 0x6f, 0xc7, - 0xbd, 0x78, 0x78, 0xf0, 0x4c, 0xab, 0x37, 0x5b, 0x8a, 0x8a, 0x10, 0xad, 0xd5, 0xad, 0xef, 0x47, - 0xa3, 0x7d, 0x47, 0x1a, 0xed, 0x83, 0xf6, 0x40, 0x53, 0xee, 0xc6, 0xa3, 0x4b, 0xc9, 0xb7, 0xd8, - 0x1b, 0x70, 0x55, 0x9e, 0x87, 0xfa, 0xcb, 0xf6, 0x60, 0x4f, 0x9c, 0xb8, 0xde, 0xe3, 0x27, 0x87, - 0x84, 0x6c, 0x34, 0x1b, 0xfc, 0x68, 0x99, 0x78, 0x31, 0x75, 0x7f, 0xa7, 0x4c, 0x8f, 0xee, 0x0a, - 0x05, 0x44, 0xfd, 0x11, 0x30, 0xf9, 0xfd, 0x49, 0xf1, 0x10, 0x13, 0x83, 0xcc, 0xd8, 0xf7, 0x26, - 0xd1, 0xa5, 0x76, 0xfc, 0x46, 0xcb, 0x79, 0x3a, 0x1b, 0xd2, 0x89, 0xe6, 0xfc, 0xd2, 0xaa, 0x0c, - 0x52, 0xff, 0x76, 0x0a, 0xaa, 0x49, 0xe5, 0x83, 0xdc, 0x9e, 0x63, 0xdd, 0xf5, 0x42, 0xfe, 0xc2, - 0x4f, 0x10, 0x3f, 0x0b, 0x39, 0xee, 0x7a, 0x21, 0x3d, 0xf1, 0x43, 0x86, 0x7c, 0xac, 0x4b, 0xf0, - 0x5c, 0xe3, 0x34, 0x6b, 0xc3, 0xa5, 0xc4, 0x13, 0x9e, 0x89, 0xf7, 0x95, 0x6a, 0xf1, 0xd3, 0x7b, - 0x0b, 0xf5, 0xd7, 0x58, 0xb0, 0xdc, 0x26, 0x71, 0xf5, 0x38, 0x33, 0xbf, 0x7a, 0xbc, 0x07, 0x95, - 0x84, 0xae, 0x43, 0xfe, 0x97, 0x71, 0xb2, 0xa6, 0x05, 0x7b, 0xfc, 0xfa, 0x6a, 0xaa, 0x7f, 0x33, - 0x05, 0x65, 0x59, 0xf3, 0xf9, 0xd6, 0x39, 0xd1, 0x35, 0x1d, 0xf1, 0xad, 0xdb, 0x66, 0xf4, 0xb2, - 0x4f, 0x04, 0x6a, 0xd3, 0x63, 0xe3, 0xdc, 0x83, 0xbc, 0x7b, 0xd2, 0x8f, 0x9b, 0x23, 0x83, 0xd8, - 0x4d, 0x00, 0xba, 0xb4, 0xb8, 0xfb, 0x1c, 0x09, 0xc4, 0x45, 0x9f, 0x39, 0x44, 0xbd, 0x05, 0xc5, - 0xdd, 0x93, 0x28, 0x94, 0x45, 0x7e, 0xe7, 0xaa, 0xc8, 0x6f, 0x3a, 0xab, 0x7f, 0x9a, 0x82, 0xea, - 0xfc, 0x4d, 0x10, 0x3a, 0x57, 0xe6, 0x4f, 0xbf, 0xf2, 0xe9, 0xb0, 0x6e, 0x0e, 0xe7, 0xef, 0x90, - 0xaf, 0xcb, 0xef, 0x90, 0xdf, 0x11, 0x99, 0xa5, 0x65, 0x91, 0x1f, 0x97, 0x25, 0xee, 0x51, 0x3f, - 0x81, 0x32, 0xfe, 0xd7, 0xac, 0xb1, 0xe5, 0xfb, 0x56, 0xf4, 0x0a, 0xee, 0x12, 0x71, 0x82, 0x88, - 0x6c, 0x3c, 0x6b, 0x2c, 0x54, 0xcd, 0x95, 0xcf, 0x96, 0xd0, 0x73, 0x3a, 0xff, 0x25, 0x0d, 0x25, - 0x49, 0x8f, 0xfc, 0x5a, 0xd3, 0xef, 0x06, 0x14, 0xe7, 0x8f, 0x68, 0x88, 0xcb, 0xab, 0x31, 0x20, - 0x31, 0x56, 0xe9, 0x85, 0xb1, 0xaa, 0x41, 0x5e, 0x04, 0xb7, 0x0a, 0xe7, 0x6f, 0x94, 0x4c, 0xba, - 0x59, 0xb3, 0xaf, 0x39, 0xe5, 0xf8, 0x00, 0xca, 0x92, 0x8f, 0x34, 0x10, 0x17, 0x3c, 0x17, 0xe9, - 0x4b, 0x73, 0x7f, 0x69, 0xc0, 0x2e, 0x43, 0x6e, 0x7c, 0xa2, 0x9b, 0x43, 0x7e, 0xab, 0xaf, 0xa8, - 0x65, 0xc7, 0x27, 0xcd, 0x21, 0x9d, 0x01, 0x8d, 0x63, 0xd5, 0x89, 0x7b, 0xae, 0x0a, 0xe3, 0x48, - 0x41, 0xba, 0x0f, 0xf9, 0xf1, 0x89, 0x7c, 0x3b, 0x6f, 0xa9, 0xcb, 0x73, 0xe3, 0x13, 0xba, 0xce, - 0xf7, 0x08, 0xb6, 0xc4, 0xfe, 0x6d, 0x04, 0x3a, 0x7f, 0x27, 0x80, 0x1e, 0x57, 0xe1, 0xaf, 0x5e, - 0x6d, 0x72, 0x5c, 0x3d, 0xe8, 0x13, 0x06, 0x67, 0x9c, 0x0a, 0x65, 0x69, 0x02, 0xf2, 0x57, 0x68, - 0x8a, 0x5a, 0x02, 0xc6, 0x9e, 0x42, 0x79, 0x7c, 0xc2, 0x07, 0x74, 0xe0, 0xed, 0x5b, 0x22, 0x94, - 0x7e, 0x6b, 0x71, 0x28, 0xe9, 0xe0, 0x3f, 0x41, 0xc9, 0xae, 0x40, 0x4e, 0x33, 0x4e, 0xfb, 0x3f, - 0xee, 0x90, 0x12, 0x59, 0xd4, 0x44, 0xea, 0x47, 0x99, 0x42, 0x55, 0xd9, 0x50, 0xff, 0x51, 0x0a, - 0xaa, 0x73, 0x1b, 0x00, 0x17, 0x21, 0x7b, 0x20, 0xbf, 0xd9, 0x5c, 0x5b, 0x34, 0x13, 0x90, 0xe4, - 0xe1, 0xe0, 0x7c, 0xca, 0x5f, 0x36, 0x5c, 0xf5, 0x72, 0xd0, 0x2a, 0xa7, 0x75, 0x7a, 0xe5, 0x6b, - 0xb1, 0xcf, 0x20, 0x3d, 0x38, 0x9f, 0x72, 0x7f, 0x13, 0x6e, 0x89, 0xdc, 0x36, 0xe5, 0x9b, 0x21, - 0x45, 0x96, 0x3c, 0x6f, 0x7d, 0xce, 0x2f, 0xdb, 0x1f, 0x68, 0xed, 0xfd, 0xba, 0xf6, 0x39, 0x05, - 0x0d, 0x91, 0xd2, 0xb0, 0xdb, 0xd3, 0x5a, 0xed, 0x67, 0x5d, 0x02, 0x64, 0xc8, 0x1b, 0x35, 0xaf, - 0x62, 0xdd, 0x34, 0x77, 0x4f, 0xe4, 0x47, 0x57, 0x52, 0x89, 0x47, 0x57, 0x92, 0xf7, 0x73, 0xd7, - 0x17, 0xef, 0xe7, 0xb2, 0x78, 0x15, 0xc6, 0x4b, 0x9a, 0xbd, 0x0d, 0x99, 0xf1, 0x89, 0x75, 0x9e, - 0x34, 0xf4, 0x92, 0x0b, 0x88, 0x08, 0xd4, 0x5f, 0xa4, 0x80, 0x25, 0x2a, 0xc2, 0x6d, 0x8f, 0x6f, - 0x5b, 0x97, 0x4f, 0xa0, 0x26, 0x9e, 0x01, 0xe4, 0x54, 0x92, 0x97, 0x5c, 0x74, 0xe9, 0x65, 0x6f, - 0x1e, 0xc5, 0x39, 0x7f, 0xdc, 0x88, 0x3d, 0x02, 0xfe, 0x0e, 0x1b, 0xc5, 0x89, 0x64, 0x2e, 0xb0, - 0x13, 0xb5, 0x39, 0xcd, 0xfc, 0xe1, 0x35, 0xf9, 0x41, 0x39, 0xee, 0x60, 0xdf, 0x98, 0x8f, 0x1a, - 0xad, 0x79, 0xf5, 0xf7, 0x53, 0x70, 0x29, 0x39, 0x21, 0x7e, 0xb9, 0x56, 0x26, 0x5f, 0xcf, 0x4b, - 0x2f, 0xbe, 0x9e, 0xb7, 0x6a, 0x3e, 0x65, 0x56, 0xce, 0xa7, 0xdf, 0x49, 0xc1, 0x96, 0xd4, 0xfb, - 0x73, 0x6b, 0xf1, 0x2f, 0xa9, 0x66, 0xd2, 0x23, 0x7a, 0x99, 0xc4, 0x23, 0x7a, 0xea, 0x1f, 0xa5, - 0xe0, 0xca, 0x42, 0x4d, 0x34, 0xeb, 0x2f, 0xb5, 0x2e, 0xc9, 0xc7, 0xf6, 0xc8, 0xc9, 0xcf, 0x43, - 0x59, 0xf9, 0xc5, 0x4e, 0x96, 0x7c, 0x3d, 0x8f, 0xae, 0x51, 0x7f, 0x08, 0x9b, 0xf3, 0x3a, 0x36, - 0xc4, 0xc3, 0x80, 0xb7, 0xa0, 0xe4, 0x5a, 0xa7, 0x7a, 0xf4, 0x6c, 0xa0, 0x88, 0x0c, 0x72, 0xad, - 0x53, 0x41, 0xa0, 0xee, 0xca, 0x02, 0x23, 0x7e, 0x43, 0xdc, 0x31, 0x13, 0x21, 0x26, 0x9e, 0x63, - 0x46, 0x28, 0xcc, 0x4d, 0x6a, 0x51, 0xde, 0xb5, 0x4e, 0x69, 0xb0, 0x4e, 0x45, 0x3e, 0x75, 0xd3, - 0x14, 0xc7, 0xec, 0xab, 0xde, 0x02, 0xba, 0x06, 0x85, 0xa9, 0x9f, 0xe8, 0x92, 0xfc, 0xd4, 0xe7, - 0xc5, 0xde, 0x15, 0x71, 0x47, 0x17, 0x1d, 0xc9, 0xf3, 0x48, 0x24, 0xf1, 0x1b, 0x03, 0x99, 0xf9, - 0x6f, 0x0c, 0x7c, 0x24, 0x64, 0x05, 0x19, 0x47, 0xbc, 0x64, 0x05, 0xd2, 0xb6, 0x79, 0x46, 0x05, - 0x57, 0x34, 0xfc, 0x24, 0x75, 0xc7, 0xfa, 0x52, 0x84, 0x3e, 0xe1, 0xa7, 0xba, 0x03, 0x25, 0x2d, - 0x61, 0x09, 0x96, 0x25, 0xa7, 0x4a, 0x90, 0x7c, 0x2e, 0x65, 0xde, 0x41, 0x5a, 0x69, 0xee, 0x53, - 0x09, 0xd4, 0x40, 0x48, 0x87, 0x17, 0x86, 0x3f, 0x3a, 0x36, 0xfc, 0x8e, 0xe5, 0x1e, 0x85, 0xc7, - 0xd8, 0xe5, 0xdc, 0xd7, 0x29, 0x77, 0x21, 0x70, 0x50, 0x34, 0xf4, 0xd8, 0x8b, 0x0e, 0x91, 0x47, - 0xaf, 0x97, 0xbb, 0xd6, 0xa9, 0xe0, 0x7f, 0x13, 0x00, 0xfb, 0x5f, 0xa0, 0xf9, 0x09, 0x5b, 0xd1, - 0x73, 0x4c, 0x8e, 0x56, 0x37, 0x45, 0x7b, 0xc5, 0xad, 0x92, 0xa6, 0x35, 0x56, 0x1d, 0x31, 0xf2, - 0xbc, 0x41, 0xa2, 0x13, 0xbe, 0xd5, 0x30, 0xb2, 0xdb, 0x50, 0x8e, 0xcc, 0x76, 0x7a, 0xa1, 0x87, - 0x17, 0x5f, 0x8a, 0x60, 0xdd, 0xd9, 0x44, 0xfd, 0x83, 0x34, 0x94, 0xeb, 0x3c, 0x08, 0x65, 0x7a, - 0xde, 0x9b, 0x86, 0xec, 0x37, 0xe1, 0x72, 0x70, 0x62, 0x4f, 0xc5, 0x73, 0xe3, 0x14, 0xfb, 0x41, - 0x71, 0xbe, 0xa2, 0x13, 0x1f, 0x48, 0x9d, 0x28, 0x58, 0x1e, 0xf6, 0x4f, 0xec, 0x29, 0x0f, 0x2f, - 0x6f, 0x9b, 0x67, 0x14, 0xcb, 0xcd, 0x8f, 0xbe, 0x59, 0xb0, 0x84, 0xa0, 0x7b, 0xa7, 0x98, 0xfd, - 0xf4, 0x44, 0x64, 0x2b, 0x4e, 0xf8, 0x11, 0x78, 0x70, 0xc2, 0x69, 0x1e, 0xc0, 0x26, 0xbf, 0x51, - 0xb2, 0xbc, 0x4b, 0x6d, 0x70, 0xc4, 0x7c, 0x7e, 0xf7, 0x61, 0x93, 0xf2, 0x13, 0xcf, 0xc0, 0xe9, - 0x23, 0x6f, 0x7a, 0x2e, 0x4e, 0xd6, 0xde, 0xbe, 0xa0, 0xaa, 0x6d, 0x4e, 0x8a, 0x20, 0xf1, 0x84, - 0x44, 0x90, 0x84, 0x5e, 0x6f, 0xc1, 0xd5, 0x0b, 0xda, 0xf4, 0xba, 0xd3, 0xfb, 0x82, 0x74, 0x7a, - 0x7f, 0x7d, 0x07, 0xb6, 0x56, 0x95, 0xf7, 0x4d, 0xf2, 0x50, 0xff, 0xb0, 0x0c, 0x30, 0x9f, 0xb1, - 0x09, 0x9d, 0x2d, 0xb5, 0xa0, 0xb3, 0x7d, 0xa3, 0xf8, 0x93, 0x0f, 0xa1, 0x8a, 0x5d, 0xa5, 0xcf, - 0x39, 0xd2, 0x2b, 0x39, 0xca, 0x48, 0x35, 0x98, 0x5f, 0xb2, 0x5b, 0x3e, 0xf1, 0xcf, 0xac, 0x3c, - 0xf1, 0xff, 0x00, 0xf2, 0xfc, 0x34, 0x2a, 0x10, 0x97, 0x34, 0xaf, 0x2e, 0xae, 0xbe, 0x87, 0x22, - 0x54, 0x3d, 0xa2, 0x63, 0x2d, 0xa8, 0xa2, 0x7c, 0xf4, 0xed, 0xf0, 0x78, 0x22, 0x5f, 0xd9, 0xbc, - 0xb9, 0xcc, 0x19, 0x91, 0xf1, 0xc7, 0xe3, 0x0c, 0x39, 0x29, 0xa9, 0x78, 0xe1, 0x44, 0xb8, 0x48, - 0x49, 0xc5, 0xcb, 0xcb, 0x2a, 0xde, 0x60, 0xc2, 0x1d, 0xa3, 0xa8, 0xe2, 0xbd, 0x07, 0x97, 0xc4, - 0xf5, 0x19, 0x64, 0xc0, 0xee, 0x24, 0x7a, 0x1e, 0x6a, 0x28, 0x5e, 0x56, 0x19, 0x4c, 0xc8, 0x00, - 0x42, 0xf2, 0xcf, 0x60, 0x6b, 0x74, 0x6c, 0xb8, 0x47, 0x96, 0x1e, 0x0e, 0x1d, 0x9d, 0x9e, 0xae, - 0xd6, 0x27, 0xc6, 0x54, 0x68, 0x9e, 0x6f, 0x2f, 0x55, 0xb6, 0x41, 0xc4, 0x83, 0xa1, 0x43, 0xb1, - 0x52, 0x71, 0x5c, 0xc8, 0xe6, 0x68, 0x11, 0xbe, 0x70, 0x3c, 0x0b, 0x4b, 0xc7, 0xb3, 0x8b, 0xba, - 0x68, 0x69, 0x85, 0x2e, 0x3a, 0xd7, 0x28, 0xcb, 0xb2, 0x46, 0xc9, 0xde, 0x85, 0xbc, 0xb8, 0x2d, - 0x28, 0x9c, 0xa3, 0x6c, 0x79, 0x75, 0x68, 0x11, 0x09, 0x96, 0x14, 0x05, 0x0b, 0xd0, 0x85, 0xe1, - 0x2a, 0x2f, 0x49, 0x86, 0x5d, 0xff, 0xaf, 0x59, 0xc8, 0x89, 0xf0, 0xdf, 0x07, 0x90, 0x31, 0x7d, - 0x6f, 0x1a, 0xc7, 0xd3, 0xae, 0x50, 0x4b, 0xe9, 0x97, 0x86, 0x50, 0x83, 0x7d, 0x08, 0x39, 0xc3, - 0x34, 0xf5, 0xf1, 0x49, 0xf2, 0xc0, 0x75, 0x41, 0x43, 0xdc, 0x5b, 0xd3, 0xb2, 0x06, 0xa9, 0x8a, - 0x9f, 0x40, 0x11, 0xe9, 0xe7, 0x51, 0x8e, 0xa5, 0x65, 0xbd, 0x37, 0xd2, 0xe5, 0xf6, 0xd6, 0xb4, - 0x82, 0x11, 0xe9, 0x75, 0xdf, 0x4f, 0xba, 0xae, 0xb9, 0xa2, 0x75, 0x7d, 0x89, 0xf5, 0x22, 0x27, - 0xf6, 0xaf, 0x03, 0xf7, 0x65, 0xc6, 0xbb, 0x6d, 0x56, 0x3e, 0xdb, 0x5b, 0xda, 0x9b, 0xf7, 0xd6, - 0x34, 0xbe, 0xe7, 0x44, 0x7b, 0xf5, 0x47, 0x91, 0x5b, 0x39, 0xfe, 0x45, 0x86, 0x15, 0x3d, 0x83, - 0x22, 0x2c, 0xf6, 0x2d, 0x93, 0x3c, 0x43, 0x36, 0xd3, 0x8c, 0x82, 0xdd, 0xf2, 0x4b, 0x6c, 0xf1, - 0x8e, 0x4c, 0x6c, 0xf1, 0xf6, 0xfc, 0x14, 0x4a, 0xdc, 0xcb, 0xc8, 0xf9, 0x0a, 0x4b, 0x5d, 0x3b, - 0xdf, 0x50, 0xe9, 0xdc, 0x6a, 0xbe, 0xbd, 0x36, 0xa2, 0x76, 0xfa, 0x96, 0x7c, 0x34, 0x70, 0x63, - 0x65, 0x47, 0x69, 0xf1, 0x29, 0x01, 0x6f, 0xac, 0xc6, 0x79, 0x58, 0x07, 0xb6, 0x84, 0x0f, 0x9d, - 0x6f, 0x9e, 0xd1, 0x7e, 0x07, 0x4b, 0xe3, 0x95, 0xd8, 0x5d, 0xf7, 0xd6, 0x34, 0x66, 0x2c, 0xef, - 0xb9, 0x0d, 0xd8, 0x8c, 0xaa, 0xc4, 0x2f, 0x61, 0xce, 0x23, 0x7a, 0xe4, 0x26, 0xcd, 0xf7, 0xcc, - 0xbd, 0x35, 0x6d, 0xc3, 0x48, 0x82, 0x58, 0x1b, 0x2e, 0x45, 0x99, 0x90, 0x2f, 0x59, 0xf4, 0x4c, - 0x79, 0x69, 0x14, 0xe5, 0x7d, 0x76, 0x6f, 0x4d, 0xdb, 0x34, 0x16, 0x81, 0xf3, 0x93, 0xf5, 0xeb, - 0x1a, 0x5c, 0x59, 0xbd, 0x9c, 0x65, 0x99, 0x9e, 0xe1, 0x32, 0x5d, 0x4d, 0xbe, 0xf0, 0x93, 0xbc, - 0x6d, 0x2f, 0x49, 0xf8, 0x1f, 0x42, 0x25, 0x21, 0xcf, 0x58, 0x09, 0xf2, 0xd1, 0x5b, 0xc0, 0x14, - 0xa3, 0xdf, 0xe8, 0x1d, 0x7c, 0xae, 0xa4, 0x10, 0xdc, 0xee, 0xf6, 0x07, 0xf5, 0xae, 0x88, 0x9b, - 0x68, 0x77, 0x45, 0xdc, 0x84, 0xfa, 0x57, 0xd3, 0x50, 0x8c, 0xcf, 0x7d, 0xbe, 0xbd, 0x7f, 0x26, - 0x76, 0x7c, 0xa4, 0x65, 0xc7, 0xc7, 0x82, 0xf1, 0xc1, 0x9f, 0xed, 0xe6, 0x2f, 0x3f, 0x6d, 0x24, - 0x55, 0xfc, 0x60, 0xf9, 0x16, 0x6f, 0xf6, 0x6b, 0xde, 0xe2, 0x95, 0x23, 0x93, 0x73, 0xc9, 0xc8, - 0xe4, 0x85, 0xf7, 0xa0, 0xf3, 0xf4, 0x52, 0xab, 0xfc, 0x1e, 0x34, 0xfd, 0x68, 0xdb, 0x0b, 0xdb, - 0x3a, 0x15, 0xa1, 0xbc, 0x22, 0x95, 0xdc, 0x0e, 0xe1, 0x35, 0xdb, 0xe1, 0xd7, 0x11, 0xad, 0x8f, - 0x61, 0x6b, 0x7c, 0x12, 0xbf, 0x0f, 0x3b, 0x37, 0xf7, 0xcb, 0x54, 0xa5, 0x95, 0x38, 0xf5, 0xff, - 0x4f, 0x01, 0xcc, 0x0f, 0x3a, 0x7e, 0x69, 0x9f, 0xa1, 0xe4, 0x96, 0x49, 0x7f, 0x85, 0x5b, 0xe6, - 0x75, 0xaf, 0xfc, 0x7c, 0x09, 0xc5, 0xf8, 0x68, 0xeb, 0xdb, 0xcf, 0x97, 0x6f, 0x54, 0xe4, 0x6f, - 0x45, 0xfe, 0xd3, 0xf8, 0x6c, 0xe8, 0x97, 0xed, 0x8b, 0x44, 0xf1, 0xe9, 0xd7, 0x14, 0x7f, 0xc6, - 0x9d, 0x98, 0x71, 0xe1, 0xbf, 0xe2, 0x45, 0x22, 0xcf, 0xdf, 0x4c, 0x62, 0xfe, 0xaa, 0x33, 0xe1, - 0x89, 0xfd, 0xe5, 0x8b, 0xfe, 0x46, 0x0d, 0xfe, 0xcf, 0xa9, 0xc8, 0x5d, 0x18, 0xbf, 0xd4, 0x7b, - 0xa1, 0xe2, 0xb8, 0xda, 0xe3, 0xf9, 0x4d, 0x8a, 0xfb, 0x4a, 0x67, 0x48, 0xe6, 0xab, 0x9c, 0x21, - 0x6f, 0x43, 0x96, 0x6f, 0x3b, 0xd9, 0x8b, 0x1c, 0x21, 0x1c, 0xff, 0xda, 0xf7, 0xf4, 0x55, 0x55, - 0x28, 0xca, 0xbc, 0xbd, 0x5b, 0x51, 0xbe, 0xd1, 0x6f, 0x01, 0xd0, 0x85, 0x88, 0xdf, 0x4e, 0x71, - 0x49, 0xf9, 0x6d, 0xfb, 0xe4, 0x57, 0xe6, 0x0d, 0xf9, 0x9f, 0x29, 0xa8, 0x24, 0x4e, 0xb5, 0xbf, - 0x45, 0x65, 0x56, 0x4a, 0xe6, 0xf4, 0xff, 0x46, 0x92, 0x39, 0x11, 0x10, 0x5a, 0x48, 0x06, 0x84, - 0xa2, 0x64, 0x2c, 0x27, 0x2c, 0x86, 0x55, 0xb6, 0x45, 0x6a, 0xa5, 0x6d, 0x71, 0x33, 0xfe, 0xc1, - 0xb0, 0x76, 0x93, 0xc7, 0x5f, 0x56, 0x34, 0x09, 0xc2, 0x3e, 0x85, 0x6b, 0xc2, 0x66, 0xe7, 0xfd, - 0xe3, 0x8d, 0xf5, 0xf8, 0xe7, 0xc4, 0x84, 0x0d, 0x7c, 0x85, 0x13, 0xf0, 0x1f, 0x4e, 0x18, 0xd7, - 0x23, 0xac, 0xda, 0x86, 0x4a, 0x22, 0x5c, 0x40, 0xfa, 0xf9, 0xc2, 0x94, 0xfc, 0xf3, 0x85, 0x6c, - 0x1b, 0xb2, 0xa7, 0xc7, 0x96, 0x6f, 0xad, 0x78, 0xf4, 0x93, 0x23, 0xd4, 0xef, 0x41, 0x59, 0x0e, - 0x5d, 0x62, 0xef, 0x42, 0xd6, 0x0e, 0xad, 0x49, 0xe4, 0x8d, 0xb8, 0xb2, 0x1c, 0xdd, 0xd4, 0x0e, - 0xad, 0x89, 0xc6, 0x89, 0xd4, 0x9f, 0xa7, 0x40, 0x59, 0xc4, 0x49, 0xbf, 0xb1, 0x98, 0xba, 0xe0, - 0x37, 0x16, 0xd7, 0x13, 0x95, 0x5c, 0xf5, 0x33, 0x89, 0xf1, 0xc3, 0x83, 0x99, 0x0b, 0x1e, 0x1e, - 0x64, 0xf7, 0xa0, 0xe0, 0x5b, 0xf4, 0x03, 0x76, 0xe6, 0x8a, 0x3b, 0x05, 0x31, 0x4e, 0xfd, 0xdd, - 0x14, 0xe4, 0x45, 0x9c, 0xd5, 0x4a, 0xf7, 0xd0, 0x77, 0x20, 0xcf, 0x7f, 0xcc, 0x2e, 0x7a, 0x70, - 0x68, 0x29, 0x68, 0x39, 0xc2, 0xb3, 0x9b, 0x3c, 0xfa, 0x2c, 0xe9, 0x2e, 0x3a, 0x70, 0x0c, 0x57, - 0x23, 0xb8, 0xf8, 0x3d, 0x14, 0x63, 0x22, 0x6e, 0x24, 0xf3, 0x67, 0x81, 0x80, 0x40, 0x74, 0xf9, - 0x58, 0xfd, 0x3e, 0xe4, 0x45, 0x1c, 0xd7, 0xca, 0xaa, 0xbc, 0xee, 0x87, 0xcc, 0xb6, 0x01, 0xe6, - 0x81, 0x5d, 0xab, 0x72, 0x50, 0x1f, 0x40, 0x21, 0x8a, 0xe5, 0xc2, 0xf9, 0x37, 0x2f, 0x5a, 0x5c, - 0x56, 0x91, 0x2b, 0xe3, 0x88, 0x87, 0xb4, 0x3b, 0xde, 0xe8, 0x84, 0x1c, 0xb8, 0x8f, 0x80, 0x6e, - 0xee, 0x0c, 0x96, 0xde, 0x4f, 0x4a, 0xbe, 0x82, 0x1e, 0x13, 0xb1, 0x07, 0x10, 0x8b, 0xd6, 0xd7, - 0x59, 0xf2, 0x6a, 0x3d, 0xba, 0xe3, 0x45, 0xb3, 0xec, 0x89, 0x70, 0x54, 0x76, 0xe8, 0xd5, 0xb1, - 0x94, 0xfc, 0x16, 0x7e, 0xa2, 0x4e, 0x9a, 0x44, 0xa6, 0x56, 0xa1, 0x2c, 0x07, 0xa0, 0xa8, 0x75, - 0xd8, 0xdc, 0xb7, 0x42, 0x03, 0xe5, 0x4f, 0xf4, 0x10, 0x0d, 0x9f, 0xbf, 0xf8, 0x91, 0x9c, 0xbf, - 0x8b, 0x74, 0x1a, 0x27, 0x52, 0x7f, 0x9e, 0x01, 0x65, 0x11, 0xf7, 0x55, 0xf7, 0xdd, 0x6e, 0x41, - 0xc9, 0xa3, 0x79, 0x91, 0xf8, 0xc5, 0x1b, 0x0e, 0x92, 0xa2, 0xcb, 0x13, 0x3f, 0x7b, 0x50, 0xb0, - 0x83, 0x3d, 0xfe, 0xc3, 0x07, 0x57, 0xf9, 0xe5, 0x26, 0xc7, 0x1b, 0xd1, 0xb4, 0x2e, 0xd3, 0x5d, - 0xa6, 0x8e, 0x37, 0xa2, 0x6b, 0x74, 0xc2, 0x19, 0xc0, 0xa3, 0x22, 0xcb, 0x5a, 0x41, 0x78, 0x00, - 0xe8, 0x4c, 0x49, 0xc4, 0x9c, 0x87, 0x81, 0xb8, 0x98, 0x58, 0xe0, 0x80, 0x41, 0x10, 0x3d, 0xd7, - 0x3c, 0x12, 0x3f, 0xcf, 0x92, 0xa6, 0xe7, 0x9a, 0x1b, 0x2e, 0xdd, 0xa2, 0xa3, 0x5f, 0x13, 0x1a, - 0x89, 0x5f, 0x7b, 0x12, 0x0f, 0x66, 0x23, 0xea, 0x0e, 0xff, 0x01, 0x1b, 0xdf, 0x0a, 0x02, 0xfe, - 0xe4, 0x5e, 0x51, 0xbc, 0x3a, 0x26, 0x80, 0xf1, 0xdb, 0x7e, 0xe2, 0xe7, 0x83, 0x90, 0x04, 0xc4, - 0xdb, 0x7e, 0xfc, 0xc7, 0x83, 0x90, 0xe0, 0x1a, 0x14, 0x7e, 0xea, 0xb9, 0x16, 0x39, 0x15, 0x4a, - 0x54, 0xab, 0x3c, 0xa6, 0xf7, 0x8d, 0xa9, 0xfa, 0x4f, 0x52, 0xb0, 0xb5, 0xd8, 0xab, 0x34, 0x61, - 0xca, 0x50, 0x68, 0xf4, 0x3a, 0x7a, 0xb7, 0xbe, 0xdf, 0x52, 0xd6, 0xd8, 0x06, 0x94, 0x7a, 0x3b, - 0x3f, 0x6a, 0x35, 0x06, 0x1c, 0x90, 0xa2, 0xdb, 0xe4, 0x7d, 0x7d, 0xaf, 0xdd, 0x6c, 0xb6, 0xba, - 0xdc, 0x7a, 0xe8, 0xed, 0xfc, 0x48, 0xef, 0xf4, 0x1a, 0xfc, 0xd7, 0x46, 0xa2, 0xf3, 0xf6, 0xbe, - 0x92, 0xa1, 0xd3, 0x78, 0x0a, 0xc2, 0xc6, 0x64, 0x96, 0xc7, 0x18, 0xbf, 0xec, 0xeb, 0x8d, 0xee, - 0x40, 0xc9, 0x61, 0xaa, 0x7b, 0xd8, 0xe9, 0x50, 0x8a, 0x82, 0x09, 0x1b, 0xbd, 0xfd, 0x03, 0xad, - 0xd5, 0xef, 0xeb, 0xfd, 0xf6, 0x4f, 0x5a, 0x4a, 0x81, 0x4a, 0xd6, 0xda, 0xcf, 0xda, 0x5d, 0x0e, - 0x28, 0xb2, 0x3c, 0xa4, 0xf7, 0xdb, 0x5d, 0x7e, 0x8b, 0x7e, 0xbf, 0xfe, 0x99, 0x52, 0xc2, 0x8f, - 0xfe, 0xe1, 0xbe, 0x52, 0x56, 0xff, 0x5d, 0x3a, 0xd2, 0x8d, 0x29, 0xb4, 0xe6, 0xeb, 0xe8, 0x83, - 0xab, 0x8e, 0xb4, 0xb6, 0x20, 0xeb, 0xd0, 0xbd, 0x60, 0xf1, 0x43, 0xaa, 0x94, 0xf8, 0x3a, 0x3f, - 0x0e, 0x79, 0x07, 0x2a, 0xf1, 0x79, 0xb4, 0xf4, 0x82, 0x72, 0x39, 0x02, 0xae, 0x70, 0xfc, 0xe7, - 0x56, 0x38, 0xfe, 0xa7, 0x76, 0x88, 0x76, 0x2f, 0x8a, 0x54, 0x3e, 0x51, 0x8a, 0x08, 0xe1, 0x3f, - 0xcb, 0xfa, 0x06, 0x50, 0x42, 0x9f, 0xb9, 0x76, 0xf4, 0xd3, 0x60, 0x05, 0x04, 0x1c, 0xba, 0x76, - 0xb8, 0x78, 0x1e, 0x5e, 0x5c, 0x3a, 0x0f, 0x97, 0xf7, 0x5e, 0x48, 0xee, 0xbd, 0xc9, 0xdf, 0xcc, - 0xe4, 0xbf, 0x09, 0x26, 0xfd, 0x66, 0xe6, 0xbb, 0xc0, 0x46, 0x33, 0x9f, 0xde, 0xb8, 0x92, 0xc8, - 0xca, 0x44, 0xa6, 0x08, 0x4c, 0xbc, 0xe9, 0xb1, 0xb7, 0x61, 0x63, 0x81, 0x9a, 0x7c, 0x44, 0x45, - 0xad, 0x9a, 0x24, 0x65, 0x0f, 0xe1, 0x92, 0x98, 0xba, 0x89, 0xbe, 0x15, 0x77, 0x0e, 0x39, 0xaa, - 0x3e, 0xef, 0x61, 0xf5, 0xd7, 0xa0, 0x10, 0x45, 0x51, 0x7d, 0xb5, 0xda, 0xbb, 0x62, 0x5c, 0xd5, - 0xbf, 0xb7, 0x0e, 0xc5, 0x38, 0xa6, 0xea, 0x6b, 0xcd, 0x0e, 0x7a, 0x2a, 0x3e, 0x38, 0x91, 0x25, - 0x48, 0x01, 0x01, 0xd1, 0x48, 0x89, 0xab, 0x3d, 0x33, 0xdf, 0x8e, 0x54, 0x37, 0x0e, 0x39, 0xf4, - 0x6d, 0x7a, 0x2c, 0xd1, 0x76, 0xa5, 0xdb, 0x81, 0x45, 0xad, 0x80, 0x00, 0x5a, 0x5d, 0xd7, 0x80, - 0xbe, 0x89, 0x33, 0xfa, 0x19, 0x51, 0xdb, 0x3d, 0x41, 0xbe, 0x0b, 0x7e, 0x46, 0x94, 0x1e, 0xba, - 0xe7, 0x01, 0x1d, 0xfc, 0x18, 0x3b, 0xfa, 0x59, 0xa6, 0x37, 0xa0, 0x38, 0x8b, 0x7f, 0xd7, 0x4b, - 0xcc, 0x88, 0x59, 0xf4, 0xab, 0x5e, 0xc9, 0x51, 0x2d, 0x2e, 0x8e, 0xea, 0xe2, 0x9c, 0x86, 0xa5, - 0x39, 0xad, 0x86, 0x90, 0x17, 0x71, 0x65, 0x5f, 0xdd, 0xe1, 0x5f, 0xd9, 0x55, 0x0a, 0xa4, 0x0d, - 0x27, 0xba, 0x92, 0x88, 0x9f, 0x0b, 0x15, 0xcb, 0x2c, 0x54, 0x4c, 0xfd, 0xeb, 0xeb, 0x00, 0xf3, - 0xf8, 0x34, 0xf6, 0xde, 0x42, 0x2c, 0x6c, 0x6a, 0x69, 0x57, 0x5f, 0x08, 0x81, 0x5d, 0x78, 0xa9, - 0x65, 0xfd, 0x6b, 0xbc, 0xd4, 0xf2, 0x18, 0x2a, 0x81, 0x3f, 0x7a, 0xad, 0xfb, 0xba, 0x14, 0xf8, - 0xa3, 0xd8, 0x7b, 0xfd, 0x08, 0x30, 0x49, 0xaf, 0xbe, 0xcd, 0x4d, 0xd6, 0x25, 0xa5, 0xa4, 0x18, - 0xf8, 0xa3, 0xde, 0xf0, 0x8b, 0x26, 0xbf, 0x4f, 0x65, 0x06, 0xa1, 0xbe, 0x4a, 0x4a, 0x6c, 0x98, - 0x41, 0xd8, 0x94, 0x05, 0xc5, 0x5d, 0xa8, 0x22, 0xed, 0x92, 0xb0, 0x28, 0x9b, 0xc1, 0xfc, 0xb8, - 0xe2, 0xc1, 0x6d, 0x28, 0xcb, 0x3f, 0x53, 0x48, 0x57, 0x29, 0x3c, 0xd7, 0xe2, 0x0f, 0xd9, 0x77, - 0x7e, 0xfa, 0xa1, 0x92, 0x7a, 0xa0, 0x42, 0x49, 0xfa, 0x19, 0x09, 0xa4, 0xd8, 0x33, 0x82, 0x63, - 0xf1, 0xa8, 0xb9, 0xe1, 0x1e, 0x59, 0x4a, 0xea, 0xc1, 0x3d, 0x54, 0x53, 0xe5, 0x1f, 0x71, 0x00, - 0xc8, 0x75, 0x3d, 0x7f, 0x62, 0x38, 0x82, 0xce, 0x9a, 0x05, 0x48, 0xf7, 0x08, 0x2e, 0xaf, 0xfc, - 0x49, 0x0a, 0xba, 0x8f, 0x63, 0x4f, 0xa6, 0x8e, 0xc5, 0xaf, 0x94, 0xec, 0x9d, 0x0f, 0x7d, 0xdb, - 0x54, 0x52, 0x0f, 0x9e, 0x46, 0xf7, 0xd3, 0xa3, 0xb2, 0x3b, 0xbd, 0x7a, 0x93, 0xef, 0x28, 0xf1, - 0x3b, 0x29, 0x83, 0x1d, 0xfe, 0x60, 0xb9, 0xd6, 0xea, 0x1f, 0x76, 0x06, 0xe2, 0x4d, 0x96, 0x07, - 0x3f, 0x84, 0xda, 0x45, 0x77, 0x2b, 0xb0, 0x46, 0x8d, 0xbd, 0x3a, 0xdd, 0x5f, 0xc1, 0x1d, 0xa4, - 0xa7, 0xf3, 0x54, 0x8a, 0x5f, 0xff, 0xe9, 0xb4, 0x28, 0x00, 0xf1, 0xc1, 0xcf, 0x52, 0x92, 0xde, - 0x14, 0xc5, 0xc7, 0xc7, 0x00, 0xd1, 0x4d, 0x32, 0x48, 0xb3, 0x0c, 0x53, 0x49, 0xb1, 0x2b, 0xc0, - 0x12, 0xa0, 0x8e, 0x37, 0x32, 0x1c, 0x65, 0x9d, 0x42, 0x0d, 0x23, 0x38, 0xdd, 0x80, 0x52, 0xd2, - 0xec, 0x4d, 0xb8, 0x16, 0xc3, 0x3a, 0xde, 0xe9, 0x81, 0x6f, 0x7b, 0xbe, 0x1d, 0x9e, 0x73, 0x74, - 0x66, 0xe7, 0x07, 0x7f, 0xf6, 0x8b, 0x9b, 0xa9, 0x7f, 0xfe, 0x8b, 0x9b, 0xa9, 0x7f, 0xff, 0x8b, - 0x9b, 0x6b, 0x3f, 0xff, 0x0f, 0x37, 0x53, 0x3f, 0x79, 0xef, 0xc8, 0x0e, 0x8f, 0x67, 0xc3, 0x87, - 0x23, 0x6f, 0xf2, 0x68, 0x62, 0x84, 0xbe, 0x7d, 0xc6, 0xc5, 0x5c, 0x94, 0x70, 0xad, 0x47, 0xd3, - 0x93, 0xa3, 0x47, 0xd3, 0xe1, 0x23, 0x9c, 0x46, 0xc3, 0xdc, 0xd4, 0xf7, 0x42, 0xef, 0xc9, 0xff, - 0x0a, 0x00, 0x00, 0xff, 0xff, 0xb8, 0xb0, 0xc8, 0x58, 0xaf, 0x81, 0x00, 0x00, + 0x96, 0x18, 0x5a, 0x2c, 0x92, 0x45, 0xf2, 0x90, 0xc5, 0xca, 0x0a, 0x95, 0x24, 0x4a, 0xad, 0x96, + 0x4a, 0x29, 0xb5, 0x5a, 0xa3, 0xee, 0x96, 0xba, 0xa5, 0x7e, 0xa8, 0x67, 0x67, 0x76, 0x86, 0xc5, + 0xa2, 0x54, 0x1c, 0xb1, 0xc8, 0x9a, 0x24, 0x4b, 0xea, 0x9e, 0xc5, 0x22, 0x91, 0x64, 0x26, 0xab, + 0xb2, 0x2b, 0x99, 0xc9, 0xce, 0x4c, 0xaa, 0xaa, 0x06, 0x58, 0xdc, 0xc1, 0xbd, 0xc0, 0xee, 0xdd, + 0x7b, 0x3f, 0x2f, 0x76, 0xaf, 0x3f, 0xbc, 0xc0, 0x78, 0x81, 0x85, 0x0d, 0xc3, 0x06, 0x6c, 0x63, + 0x8d, 0xb5, 0xbf, 0xfd, 0xb3, 0xb6, 0xe1, 0x85, 0x01, 0x03, 0x36, 0x6c, 0x03, 0x6b, 0x63, 0x0c, + 0x7f, 0xfa, 0x05, 0x1b, 0xf0, 0xa7, 0x8d, 0x73, 0x22, 0x22, 0x19, 0x49, 0x52, 0xad, 0xee, 0x9e, + 0x59, 0xc0, 0xfe, 0xa9, 0xca, 0x38, 0x8f, 0x78, 0xc7, 0x89, 0x73, 0x4e, 0x9c, 0x08, 0x02, 0x4c, + 0x3c, 0xcb, 0xbf, 0x3f, 0x09, 0x83, 0x38, 0x60, 0x39, 0xfc, 0xbe, 0xfa, 0xde, 0x91, 0x1b, 0x1f, + 0x4f, 0x07, 0xf7, 0x87, 0xc1, 0xf8, 0xc1, 0x51, 0x70, 0x14, 0x3c, 0x20, 0xe4, 0x60, 0x3a, 0xa2, + 0x14, 0x25, 0xe8, 0x8b, 0x33, 0x5d, 0x05, 0x2f, 0x18, 0x9e, 0x88, 0xef, 0x8d, 0xd8, 0x1d, 0x3b, + 0x51, 0x6c, 0x8d, 0x27, 0x1c, 0xa0, 0xff, 0x49, 0x06, 0x72, 0xfd, 0xf3, 0x89, 0xc3, 0xaa, 0xb0, + 0xea, 0xda, 0xb5, 0xcc, 0x76, 0xe6, 0x6e, 0xde, 0x58, 0x75, 0x6d, 0xb6, 0x0d, 0x65, 0x3f, 0x88, + 0x3b, 0x53, 0xcf, 0xb3, 0x06, 0x9e, 0x53, 0x5b, 0xdd, 0xce, 0xdc, 0x2d, 0x1a, 0x2a, 0x88, 0xbd, + 0x01, 0x25, 0x6b, 0x1a, 0x07, 0xa6, 0xeb, 0x0f, 0xc3, 0x5a, 0x96, 0xf0, 0x45, 0x04, 0xb4, 0xfc, + 0x61, 0xc8, 0xb6, 0x20, 0x7f, 0xea, 0xda, 0xf1, 0x71, 0x2d, 0x47, 0x39, 0xf2, 0x04, 0x42, 0xa3, + 0xa1, 0xe5, 0x39, 0xb5, 0x3c, 0x87, 0x52, 0x02, 0xa1, 0x31, 0x15, 0xb2, 0xb6, 0x9d, 0xb9, 0x5b, + 0x32, 0x78, 0x82, 0x5d, 0x07, 0x70, 0xfc, 0xe9, 0xf8, 0xa5, 0xe5, 0x4d, 0x9d, 0xa8, 0x56, 0x20, + 0x94, 0x02, 0xd1, 0x7f, 0x00, 0xa5, 0x71, 0x74, 0xb4, 0xe7, 0x58, 0xb6, 0x13, 0xb2, 0xcb, 0x50, + 0x18, 0x47, 0x47, 0x66, 0x6c, 0x1d, 0x89, 0x26, 0xac, 0x8d, 0xa3, 0xa3, 0xbe, 0x75, 0xc4, 0xae, + 0x40, 0x91, 0x10, 0xe7, 0x13, 0xde, 0x86, 0xbc, 0x81, 0x84, 0xd8, 0x62, 0xfd, 0x77, 0xd7, 0xa0, + 0xd0, 0x76, 0x63, 0x27, 0xb4, 0x3c, 0x76, 0x09, 0xd6, 0xdc, 0xc8, 0x9f, 0x7a, 0x1e, 0xb1, 0x17, + 0x0d, 0x91, 0x62, 0x97, 0x20, 0xef, 0x3e, 0x7e, 0x69, 0x79, 0x9c, 0x77, 0x6f, 0xc5, 0xe0, 0x49, + 0x56, 0x83, 0x35, 0xf7, 0x83, 0x8f, 0x11, 0x91, 0x15, 0x08, 0x91, 0x26, 0xcc, 0xa3, 0x87, 0x88, + 0xc9, 0x25, 0x18, 0x4a, 0x13, 0xe6, 0xe3, 0x0f, 0x11, 0x83, 0xad, 0xcf, 0x12, 0x86, 0xd2, 0x58, + 0xca, 0x94, 0x4a, 0xc1, 0x0e, 0x58, 0xc7, 0x52, 0xa6, 0xb2, 0x94, 0x29, 0x2f, 0xa5, 0x20, 0x10, + 0x22, 0x4d, 0x18, 0x5e, 0x4a, 0x31, 0xc1, 0x24, 0xa5, 0x4c, 0x79, 0x29, 0xa5, 0xed, 0xcc, 0xdd, + 0x1c, 0x61, 0x78, 0x29, 0x5b, 0x90, 0xb3, 0x11, 0x0e, 0xdb, 0x99, 0xbb, 0x99, 0xbd, 0x15, 0x83, + 0x52, 0x08, 0x8d, 0x10, 0x5a, 0xc6, 0x0e, 0x46, 0x68, 0x24, 0xa0, 0x03, 0x84, 0x56, 0xb0, 0x37, + 0x10, 0x3a, 0x10, 0xd0, 0x11, 0x42, 0xd7, 0xb7, 0x33, 0x77, 0x57, 0x11, 0x8a, 0x29, 0x76, 0x15, + 0x0a, 0xb6, 0x15, 0x3b, 0x88, 0xa8, 0x8a, 0x26, 0x4b, 0x00, 0xe2, 0x70, 0xc6, 0x21, 0x6e, 0x43, + 0x34, 0x5a, 0x02, 0x98, 0x0e, 0x65, 0x24, 0x93, 0x78, 0x4d, 0xe0, 0x55, 0x20, 0xfb, 0x08, 0x2a, + 0xb6, 0x33, 0x74, 0xc7, 0x96, 0xc7, 0xdb, 0xb4, 0xb9, 0x9d, 0xb9, 0x5b, 0x7e, 0xb8, 0x71, 0x9f, + 0xd6, 0x44, 0x82, 0xd9, 0x5b, 0x31, 0x52, 0x64, 0xec, 0x31, 0xac, 0x8b, 0xf4, 0x07, 0x0f, 0xa9, + 0x63, 0x19, 0xf1, 0x69, 0x29, 0xbe, 0x0f, 0x1e, 0x3e, 0xde, 0x5b, 0x31, 0xd2, 0x84, 0xec, 0x36, + 0x54, 0x92, 0x25, 0x82, 0x8c, 0x17, 0x44, 0xad, 0x52, 0x50, 0x6c, 0xd6, 0x17, 0x51, 0xe0, 0x23, + 0xc1, 0x96, 0xe8, 0x37, 0x09, 0x60, 0xdb, 0x00, 0xb6, 0x33, 0xb2, 0xa6, 0x5e, 0x8c, 0xe8, 0x8b, + 0xa2, 0x03, 0x15, 0x18, 0xbb, 0x0e, 0xa5, 0xe9, 0x04, 0x5b, 0xf9, 0xdc, 0xf2, 0x6a, 0x97, 0x04, + 0xc1, 0x0c, 0x84, 0xb9, 0xe3, 0x3c, 0x47, 0xec, 0x65, 0x31, 0xba, 0x12, 0x80, 0xc3, 0xfb, 0xd2, + 0x19, 0x22, 0xaa, 0x26, 0x0a, 0x16, 0x69, 0x5c, 0x45, 0x6e, 0xb4, 0xe3, 0xfa, 0xb5, 0x2b, 0x34, + 0x83, 0x79, 0x82, 0x5d, 0x83, 0x6c, 0x14, 0x0e, 0x6b, 0x57, 0xa9, 0xfd, 0xc0, 0xdb, 0xdf, 0x3c, + 0x9b, 0x84, 0x06, 0x82, 0x77, 0x0a, 0x90, 0xa7, 0xd5, 0xa4, 0x5f, 0x83, 0xe2, 0x81, 0x15, 0x5a, + 0x63, 0xc3, 0x19, 0x31, 0x0d, 0xb2, 0x93, 0x20, 0x12, 0xeb, 0x08, 0x3f, 0xf5, 0x36, 0xac, 0x3d, + 0xb7, 0x42, 0xc4, 0x31, 0xc8, 0xf9, 0xd6, 0xd8, 0x21, 0x64, 0xc9, 0xa0, 0x6f, 0x5c, 0x3b, 0xd1, + 0x79, 0x14, 0x3b, 0x63, 0x21, 0x24, 0x44, 0x0a, 0xe1, 0x47, 0x5e, 0x30, 0x10, 0x6b, 0xa4, 0x68, + 0x88, 0x94, 0xfe, 0x7f, 0x66, 0x60, 0xad, 0x11, 0x78, 0x98, 0xdd, 0x65, 0x28, 0x84, 0x8e, 0x67, + 0xce, 0x8a, 0x5b, 0x0b, 0x1d, 0xef, 0x20, 0x88, 0x10, 0x31, 0x0c, 0x38, 0x82, 0xaf, 0xda, 0xb5, + 0x61, 0x40, 0x08, 0x59, 0x81, 0xac, 0x52, 0x81, 0x2b, 0x50, 0x8c, 0x07, 0x9e, 0x49, 0xf0, 0x1c, + 0xc1, 0x0b, 0xf1, 0xc0, 0xeb, 0x20, 0xea, 0x32, 0x14, 0xec, 0x01, 0xc7, 0xe4, 0x09, 0xb3, 0x66, + 0x0f, 0x10, 0xa1, 0x7f, 0x0a, 0x25, 0xc3, 0x3a, 0x15, 0xd5, 0xb8, 0x08, 0x6b, 0x98, 0x81, 0x90, + 0x7f, 0x39, 0x23, 0x1f, 0x0f, 0xbc, 0x96, 0x8d, 0x60, 0xac, 0x84, 0x6b, 0x53, 0x1d, 0x72, 0x46, + 0x7e, 0x18, 0x78, 0x2d, 0x5b, 0xef, 0x03, 0x34, 0x82, 0x30, 0xfc, 0xd6, 0x4d, 0xd8, 0x82, 0xbc, + 0xed, 0x4c, 0xe2, 0x63, 0x2e, 0x3a, 0x0c, 0x9e, 0xd0, 0xef, 0x41, 0x11, 0xc7, 0xa5, 0xed, 0x46, + 0x31, 0xbb, 0x0e, 0x39, 0xcf, 0x8d, 0xe2, 0x5a, 0x66, 0x3b, 0x3b, 0x37, 0x6a, 0x04, 0xd7, 0xb7, + 0xa1, 0xb8, 0x6f, 0x9d, 0x3d, 0xc7, 0x91, 0xc3, 0xdc, 0x68, 0x08, 0xc5, 0x90, 0x88, 0xf1, 0xac, + 0x00, 0xf4, 0xad, 0xf0, 0xc8, 0x89, 0x49, 0xd2, 0xfd, 0xd7, 0x0c, 0x94, 0x7b, 0xd3, 0xc1, 0x97, + 0x53, 0x27, 0x3c, 0xc7, 0x3a, 0xdf, 0x85, 0x6c, 0x7c, 0x3e, 0x21, 0x8e, 0xea, 0xc3, 0x4b, 0x3c, + 0x7b, 0x05, 0x7f, 0x1f, 0x99, 0x0c, 0x24, 0xc1, 0x46, 0xf8, 0x81, 0xed, 0xc8, 0x3e, 0xc8, 0x1b, + 0x6b, 0x98, 0x6c, 0xd9, 0xb8, 0x5d, 0x04, 0x13, 0x31, 0x0a, 0xab, 0xc1, 0x84, 0x6d, 0x43, 0x7e, + 0x78, 0xec, 0x7a, 0x36, 0x0d, 0x40, 0xba, 0xce, 0x1c, 0x81, 0xa3, 0x14, 0x06, 0xa7, 0x66, 0xe4, + 0xfe, 0x54, 0x8a, 0xff, 0x42, 0x18, 0x9c, 0xf6, 0xdc, 0x9f, 0x3a, 0x7a, 0x5f, 0xec, 0x41, 0x00, + 0x6b, 0xbd, 0x46, 0xbd, 0x5d, 0x37, 0xb4, 0x15, 0xfc, 0x6e, 0x7e, 0xd6, 0xea, 0xf5, 0x7b, 0x5a, + 0x86, 0x55, 0x01, 0x3a, 0xdd, 0xbe, 0x29, 0xd2, 0xab, 0x6c, 0x0d, 0x56, 0x5b, 0x1d, 0x2d, 0x8b, + 0x34, 0x08, 0x6f, 0x75, 0xb4, 0x1c, 0x2b, 0x40, 0xb6, 0xde, 0xf9, 0x5c, 0xcb, 0xd3, 0x47, 0xbb, + 0xad, 0xad, 0xe9, 0x7f, 0xb6, 0x0a, 0xa5, 0xee, 0xe0, 0x0b, 0x67, 0x18, 0x63, 0x9b, 0x71, 0x96, + 0x3a, 0xe1, 0x4b, 0x27, 0xa4, 0x66, 0x67, 0x0d, 0x91, 0xc2, 0x86, 0xd8, 0x03, 0x6a, 0x5c, 0xd6, + 0x58, 0xb5, 0x07, 0x44, 0x37, 0x3c, 0x76, 0xc6, 0x16, 0x35, 0x0e, 0xe9, 0x28, 0x85, 0xab, 0x22, + 0x18, 0x7c, 0x41, 0xcd, 0xcb, 0x1a, 0xf8, 0xc9, 0x6e, 0x40, 0x99, 0xe7, 0xa1, 0xce, 0x2f, 0xe0, + 0xa0, 0xf9, 0xc9, 0xb7, 0xa6, 0x4e, 0x3e, 0xe2, 0xa4, 0x5c, 0x39, 0x52, 0xec, 0x6d, 0x1c, 0xd4, + 0x11, 0x33, 0x3a, 0x18, 0x7c, 0xc1, 0xb1, 0x45, 0x3e, 0xa3, 0x83, 0xc1, 0x17, 0x84, 0x7a, 0x07, + 0x36, 0xa3, 0xe9, 0x20, 0x1a, 0x86, 0xee, 0x24, 0x76, 0x03, 0x9f, 0xd3, 0x94, 0x88, 0x46, 0x53, + 0x11, 0x44, 0x7c, 0x17, 0x8a, 0x93, 0xe9, 0xc0, 0x74, 0xfd, 0x51, 0x40, 0x62, 0xbf, 0xfc, 0x70, + 0x9d, 0x0f, 0xcc, 0xc1, 0x74, 0xd0, 0xf2, 0x47, 0x81, 0x51, 0x98, 0xf0, 0x0f, 0xa6, 0xc3, 0xba, + 0x1f, 0xc4, 0x26, 0xaa, 0x0a, 0xe6, 0xd8, 0x89, 0x2d, 0xda, 0x0f, 0xf8, 0x86, 0xdf, 0x0e, 0x86, + 0x27, 0xfb, 0x4e, 0x6c, 0xe9, 0x77, 0xa0, 0x20, 0xf8, 0x70, 0xef, 0x8f, 0x1d, 0xdf, 0xf2, 0x63, + 0x33, 0x51, 0x1a, 0x8a, 0x1c, 0xd0, 0xb2, 0xf5, 0x3f, 0xce, 0x80, 0xd6, 0x53, 0xaa, 0x82, 0xcc, + 0x4b, 0x25, 0xc7, 0x9b, 0x00, 0xd6, 0x70, 0x18, 0x4c, 0x79, 0x36, 0x7c, 0x82, 0x95, 0x04, 0xa4, + 0x65, 0xab, 0xfd, 0x97, 0x4d, 0xf5, 0xdf, 0x4d, 0xa8, 0x48, 0x3e, 0x65, 0xd1, 0x97, 0x05, 0x4c, + 0xf6, 0x60, 0x34, 0x4d, 0xad, 0xfc, 0x42, 0x34, 0xe5, 0xdc, 0x97, 0x60, 0x8d, 0x34, 0x8c, 0x48, + 0x8e, 0x0a, 0x4f, 0xe9, 0xff, 0x22, 0x03, 0xeb, 0x2d, 0xdf, 0x76, 0xce, 0x7a, 0x43, 0xcb, 0x97, + 0x9d, 0xe2, 0x46, 0xa6, 0x8b, 0x30, 0x33, 0x1a, 0x5a, 0xbe, 0x50, 0x0e, 0xca, 0x6e, 0x94, 0xd0, + 0x61, 0x1b, 0x38, 0x01, 0x15, 0xb5, 0x4a, 0x39, 0x96, 0x08, 0x42, 0x85, 0xdd, 0x81, 0x8d, 0x81, + 0xe3, 0x05, 0xfe, 0x91, 0x19, 0x07, 0x26, 0xd7, 0x72, 0x78, 0x5b, 0xd6, 0x39, 0xb8, 0x1f, 0xf4, + 0x49, 0xdb, 0xd9, 0x82, 0xfc, 0xc4, 0x0a, 0xe3, 0xa8, 0x96, 0xdb, 0xce, 0xe2, 0x32, 0xa6, 0x04, + 0x76, 0xb3, 0x1b, 0x99, 0x53, 0xdf, 0xfd, 0x72, 0xca, 0x9b, 0x51, 0x34, 0x8a, 0x6e, 0x74, 0x48, + 0x69, 0x76, 0x17, 0x34, 0x5e, 0x32, 0x65, 0xab, 0xce, 0xb3, 0x2a, 0xc1, 0x29, 0x63, 0x12, 0x76, + 0xff, 0xcf, 0x2a, 0x14, 0x9f, 0x4c, 0xfd, 0x21, 0x0e, 0x06, 0xbb, 0x05, 0xb9, 0xd1, 0xd4, 0x1f, + 0x52, 0x5b, 0x92, 0xad, 0x34, 0x59, 0x27, 0x06, 0x21, 0x51, 0x02, 0x59, 0xe1, 0x11, 0x4a, 0xae, + 0x05, 0x09, 0x84, 0x70, 0xfd, 0xef, 0x65, 0x78, 0x8e, 0x4f, 0x3c, 0xeb, 0x88, 0x15, 0x21, 0xd7, + 0xe9, 0x76, 0x9a, 0xda, 0x0a, 0xab, 0x40, 0xb1, 0xd5, 0xe9, 0x37, 0x8d, 0x4e, 0xbd, 0xad, 0x65, + 0x68, 0x39, 0xf7, 0xeb, 0x3b, 0xed, 0xa6, 0xb6, 0x8a, 0x98, 0xe7, 0xdd, 0x76, 0xbd, 0xdf, 0x6a, + 0x37, 0xb5, 0x1c, 0xc7, 0x18, 0xad, 0x46, 0x5f, 0x2b, 0x32, 0x0d, 0x2a, 0x07, 0x46, 0x77, 0xf7, + 0xb0, 0xd1, 0x34, 0x3b, 0x87, 0xed, 0xb6, 0xa6, 0xb1, 0x0b, 0xb0, 0x91, 0x40, 0xba, 0x1c, 0xb8, + 0x8d, 0x2c, 0xcf, 0xeb, 0x46, 0xdd, 0x78, 0xaa, 0xfd, 0x90, 0x15, 0x21, 0x5b, 0x7f, 0xfa, 0x54, + 0xfb, 0x19, 0x4a, 0x86, 0xd2, 0x8b, 0x56, 0xc7, 0x7c, 0x5e, 0x6f, 0x1f, 0x36, 0xb5, 0x9f, 0xad, + 0xca, 0x74, 0xd7, 0xd8, 0x6d, 0x1a, 0xda, 0xcf, 0x72, 0x6c, 0x13, 0x2a, 0x3f, 0xe9, 0x76, 0x9a, + 0xfb, 0xf5, 0x83, 0x03, 0xaa, 0xc8, 0xcf, 0x8a, 0xfa, 0x7f, 0xc8, 0x41, 0x0e, 0x5b, 0xc2, 0xf4, + 0x99, 0x14, 0x4c, 0x9a, 0x88, 0x62, 0x68, 0x27, 0xf7, 0xa7, 0x7f, 0x7e, 0x63, 0x85, 0xcb, 0xbf, + 0x9b, 0x90, 0xf5, 0xdc, 0x98, 0x86, 0x35, 0x59, 0x3b, 0x42, 0x67, 0xdc, 0x5b, 0x31, 0x10, 0xc7, + 0xae, 0x43, 0x86, 0x0b, 0xc2, 0xf2, 0xc3, 0xaa, 0x58, 0x5c, 0x62, 0x27, 0xdd, 0x5b, 0x31, 0x32, + 0x13, 0x76, 0x0d, 0x32, 0x2f, 0x85, 0x54, 0xac, 0x70, 0x3c, 0xdf, 0x4b, 0x11, 0xfb, 0x92, 0x6d, + 0x43, 0x76, 0x18, 0x70, 0x8d, 0x30, 0xc1, 0xf3, 0x9d, 0x05, 0xf3, 0x1f, 0x06, 0x1e, 0xbb, 0x05, + 0xd9, 0xd0, 0x3a, 0xa5, 0x91, 0x4d, 0x86, 0x2b, 0xd9, 0xba, 0x90, 0x28, 0xb4, 0x4e, 0xb1, 0x12, + 0x23, 0x92, 0x23, 0x49, 0x25, 0xe4, 0x78, 0x63, 0x31, 0x23, 0xb6, 0x0d, 0x99, 0x53, 0x92, 0x24, + 0x89, 0x12, 0xf4, 0xc2, 0xf5, 0xed, 0xe0, 0xb4, 0x37, 0x71, 0x86, 0x48, 0x71, 0xca, 0xde, 0x82, + 0x6c, 0x34, 0x1d, 0x90, 0x24, 0x29, 0x3f, 0xdc, 0x5c, 0xd8, 0x13, 0xb0, 0xa0, 0x68, 0x3a, 0x60, + 0x77, 0x20, 0x37, 0x0c, 0xc2, 0x50, 0x48, 0x13, 0x4d, 0x56, 0x58, 0x6e, 0x87, 0xa8, 0x14, 0x22, + 0x1e, 0x0b, 0x8c, 0x49, 0x86, 0x24, 0x44, 0xb3, 0xfd, 0x08, 0x0b, 0x8c, 0xd9, 0x6d, 0xb1, 0xc9, + 0x55, 0xd4, 0x5a, 0xcb, 0x2d, 0x10, 0xf3, 0x41, 0x2c, 0x0e, 0xd2, 0xd8, 0x3a, 0x23, 0x8d, 0x33, + 0x21, 0x92, 0x7b, 0x1f, 0xd6, 0x69, 0x6c, 0x9d, 0xb1, 0xdb, 0x90, 0x7d, 0xe9, 0x0c, 0x49, 0xf9, + 0x4c, 0x4a, 0x13, 0x83, 0xf4, 0x9c, 0x9a, 0x87, 0x68, 0x9a, 0xf7, 0x81, 0x67, 0x93, 0x1e, 0x9a, + 0x8c, 0xe5, 0x93, 0xc0, 0xb3, 0x9f, 0xd3, 0x58, 0x12, 0x12, 0xb7, 0x7c, 0x6b, 0x7a, 0x86, 0xd2, + 0x48, 0xe3, 0x9b, 0xb3, 0x35, 0x3d, 0x6b, 0xd9, 0x28, 0xfc, 0x7d, 0xfb, 0x25, 0x69, 0x9f, 0x19, + 0x03, 0x3f, 0xd1, 0x3c, 0x8a, 0x1c, 0xcf, 0x19, 0xc6, 0xee, 0x4b, 0x37, 0x3e, 0x27, 0xfd, 0x32, + 0x63, 0xa8, 0xa0, 0x9d, 0x35, 0xc8, 0x39, 0x67, 0x93, 0x50, 0xdf, 0x83, 0x82, 0x28, 0x65, 0xc1, + 0xc6, 0xba, 0x02, 0x45, 0x37, 0x32, 0x87, 0x81, 0x1f, 0xc5, 0x42, 0x77, 0x2a, 0xb8, 0x51, 0x03, + 0x93, 0x28, 0x2e, 0x6d, 0x2b, 0xe6, 0x9b, 0x50, 0xc5, 0xa0, 0x6f, 0xfd, 0x21, 0xc0, 0xac, 0x59, + 0x58, 0x27, 0xcf, 0xf1, 0xa5, 0x9a, 0xe6, 0x39, 0x7e, 0xc2, 0xb3, 0xaa, 0xf0, 0x5c, 0x81, 0x52, + 0xa2, 0x19, 0xb3, 0x0a, 0x64, 0x2c, 0xb1, 0xfd, 0x65, 0x2c, 0xfd, 0x2e, 0x2a, 0xaa, 0x52, 0xf7, + 0x4d, 0xe3, 0x30, 0x25, 0x37, 0xc5, 0xcc, 0x40, 0xff, 0x1e, 0x54, 0x0c, 0x27, 0x9a, 0x7a, 0x71, + 0x23, 0xf0, 0x76, 0x9d, 0x11, 0x7b, 0x17, 0x20, 0x49, 0x47, 0x42, 0x4b, 0x99, 0xcd, 0xdd, 0x5d, + 0x67, 0x64, 0x28, 0x78, 0xfd, 0xaf, 0xe5, 0x48, 0xdf, 0xdb, 0xe5, 0x8a, 0x96, 0xd0, 0xa8, 0x32, + 0x8a, 0x46, 0x95, 0xec, 0x0d, 0xab, 0x69, 0xad, 0xf2, 0xd8, 0xb5, 0x6d, 0xc7, 0x97, 0xda, 0x23, + 0x4f, 0xe1, 0x60, 0x5b, 0xde, 0x11, 0x2d, 0xa8, 0xea, 0x43, 0x26, 0x0b, 0x1d, 0x4f, 0x42, 0x27, + 0x8a, 0xb8, 0xde, 0x62, 0x79, 0x47, 0x72, 0x6d, 0xe7, 0xbf, 0x6a, 0x6d, 0x5f, 0x81, 0x22, 0x6e, + 0x79, 0x64, 0xf5, 0xad, 0xf1, 0xde, 0x17, 0xe6, 0x2d, 0x7b, 0x1b, 0x0a, 0x42, 0x5f, 0x17, 0x8b, + 0x4a, 0x4c, 0x97, 0x5d, 0x0e, 0x34, 0x24, 0x96, 0xd5, 0x50, 0xc9, 0x1b, 0x8f, 0x1d, 0x3f, 0x96, + 0xfb, 0xb4, 0x48, 0xb2, 0x77, 0xa0, 0x14, 0xf8, 0x26, 0x57, 0xea, 0xc5, 0xaa, 0x12, 0xd3, 0xb7, + 0xeb, 0x1f, 0x12, 0xd4, 0x28, 0x06, 0xe2, 0x0b, 0xab, 0xe2, 0x05, 0xa7, 0xe6, 0xd0, 0x0a, 0x6d, + 0x5a, 0x59, 0x45, 0xa3, 0xe0, 0x05, 0xa7, 0x0d, 0x2b, 0xb4, 0xb9, 0xde, 0xf2, 0xa5, 0x3f, 0x1d, + 0xd3, 0x6a, 0x5a, 0x37, 0x44, 0x8a, 0x5d, 0x83, 0xd2, 0xd0, 0x9b, 0x46, 0xb1, 0x13, 0xee, 0x9c, + 0x73, 0x33, 0xcd, 0x98, 0x01, 0xb0, 0x5e, 0x93, 0xd0, 0x1d, 0x5b, 0xe1, 0x39, 0x2d, 0x9d, 0xa2, + 0x21, 0x93, 0xb4, 0xd1, 0x9c, 0xb8, 0xf6, 0x19, 0xb7, 0xd5, 0x0c, 0x9e, 0x40, 0xfa, 0x63, 0xb2, + 0xa4, 0x23, 0x5a, 0x1f, 0x45, 0x43, 0x26, 0x69, 0x1c, 0xe8, 0x93, 0x56, 0x44, 0xc9, 0x10, 0xa9, + 0x94, 0xd2, 0xbd, 0xf9, 0x4a, 0xa5, 0x9b, 0xcd, 0xeb, 0x3d, 0x41, 0xe8, 0x1e, 0xb9, 0x42, 0x6b, + 0xb9, 0xc0, 0xf5, 0x1e, 0x0e, 0xa2, 0x8d, 0xea, 0x4b, 0x28, 0x88, 0x2e, 0xc6, 0x1d, 0x08, 0x97, + 0x4f, 0x5a, 0x3c, 0xf3, 0x1d, 0x08, 0xe1, 0xec, 0x16, 0xac, 0x8b, 0xbc, 0xa2, 0x38, 0x74, 0xfd, + 0x23, 0x31, 0x79, 0x2a, 0x1c, 0xd8, 0x23, 0x18, 0x2a, 0x0a, 0x38, 0xbc, 0xa6, 0x35, 0x70, 0x3d, + 0x5c, 0xa6, 0x59, 0xa1, 0xd4, 0x4c, 0x3d, 0xaf, 0xce, 0x41, 0x7a, 0x17, 0x8a, 0x72, 0x40, 0x7e, + 0x25, 0x65, 0xea, 0xbf, 0x9d, 0x81, 0x32, 0xa9, 0x07, 0x5d, 0x52, 0x7e, 0xd8, 0xbb, 0xc0, 0x86, + 0xa1, 0x63, 0xc5, 0x8e, 0xe9, 0x9c, 0xc5, 0xa1, 0x25, 0x94, 0x00, 0xae, 0x49, 0x68, 0x1c, 0xd3, + 0x44, 0x04, 0xd7, 0x03, 0x6e, 0x40, 0x79, 0x62, 0x85, 0x91, 0x54, 0x2a, 0x79, 0x01, 0xc0, 0x41, + 0x42, 0xa5, 0xd3, 0xfc, 0xa3, 0xd0, 0x1a, 0x9b, 0x71, 0x70, 0xe2, 0xf8, 0x5c, 0x9d, 0xe6, 0x86, + 0x44, 0x95, 0xe0, 0x7d, 0x04, 0x93, 0x56, 0xfd, 0xaf, 0x32, 0xb0, 0x7e, 0xc0, 0x47, 0xfd, 0x99, + 0x73, 0xbe, 0xcb, 0xad, 0xb7, 0xa1, 0x5c, 0xb1, 0x39, 0x83, 0xbe, 0xd9, 0x75, 0x28, 0x4f, 0x4e, + 0x9c, 0x73, 0x33, 0x65, 0xe9, 0x94, 0x10, 0xd4, 0xa0, 0xb5, 0xf9, 0x1d, 0x58, 0x0b, 0xa8, 0x21, + 0x62, 0x8f, 0x13, 0x5b, 0x83, 0xd2, 0x42, 0x43, 0x10, 0xa0, 0xba, 0x94, 0x64, 0xa5, 0xea, 0x65, + 0x22, 0x33, 0xaa, 0xfe, 0x16, 0xe4, 0x11, 0x15, 0xd5, 0xf2, 0x5c, 0xcf, 0xa1, 0x04, 0x7b, 0x1f, + 0xd6, 0x87, 0xc1, 0x78, 0x62, 0x4a, 0x76, 0xb1, 0xdb, 0xa5, 0x65, 0x4a, 0x19, 0x49, 0x0e, 0x78, + 0x5e, 0xfa, 0xef, 0x67, 0xa1, 0x48, 0x75, 0x10, 0x62, 0xc5, 0xb5, 0xcf, 0xa4, 0x58, 0x29, 0x19, + 0x79, 0xd7, 0x46, 0xa9, 0xfd, 0x1a, 0xd5, 0x2c, 0x51, 0xb9, 0xb2, 0xaa, 0xca, 0x75, 0x09, 0xd6, + 0x84, 0xbe, 0x95, 0xe3, 0x72, 0x67, 0xfa, 0x6a, 0x6d, 0x2b, 0xbf, 0x4c, 0xdb, 0xc2, 0x21, 0xe4, + 0x34, 0xce, 0x19, 0xee, 0x6f, 0x5c, 0xb4, 0x00, 0x81, 0x9a, 0x08, 0x51, 0x85, 0x46, 0x21, 0x2d, + 0x34, 0x6a, 0x50, 0x78, 0xe9, 0x46, 0x2e, 0x4e, 0x90, 0x22, 0x5f, 0x86, 0x22, 0xa9, 0x0c, 0x43, + 0xe9, 0x75, 0xc3, 0x90, 0x34, 0xdb, 0xf2, 0x8e, 0xb8, 0xda, 0x2f, 0x9b, 0x5d, 0xf7, 0x8e, 0x02, + 0xf6, 0x01, 0x5c, 0x9c, 0xa1, 0x45, 0x6b, 0xc8, 0x3d, 0x46, 0x1e, 0x20, 0x83, 0x25, 0x94, 0xd4, + 0x22, 0xb2, 0xcb, 0xee, 0xc1, 0xa6, 0xc2, 0x32, 0x41, 0xf5, 0x26, 0x22, 0x99, 0x53, 0x32, 0x36, + 0x12, 0x72, 0xd2, 0x7a, 0x22, 0xfd, 0x1f, 0xae, 0xc2, 0xfa, 0x93, 0x20, 0x74, 0xdc, 0x23, 0x7f, + 0x36, 0xeb, 0x16, 0x34, 0x7f, 0x39, 0x13, 0x57, 0x95, 0x99, 0x78, 0x03, 0xca, 0x23, 0xce, 0x68, + 0xc6, 0x03, 0xee, 0x34, 0xc8, 0x19, 0x20, 0x40, 0xfd, 0x81, 0x87, 0xab, 0x59, 0x12, 0x10, 0x73, + 0x8e, 0x98, 0x25, 0x13, 0xee, 0x35, 0xec, 0xbb, 0x24, 0x75, 0x6d, 0xc7, 0x73, 0x62, 0x3e, 0x3c, + 0xd5, 0x87, 0x6f, 0xca, 0x9d, 0x5e, 0xa9, 0xd3, 0x7d, 0xc3, 0x19, 0xd5, 0x49, 0x3d, 0x42, 0x21, + 0xbc, 0x4b, 0xe4, 0x82, 0x57, 0x48, 0xec, 0xb5, 0xaf, 0xc9, 0xcb, 0x25, 0x87, 0xde, 0x87, 0x52, + 0x02, 0x46, 0x5d, 0xd7, 0x68, 0x0a, 0xfd, 0x76, 0x85, 0x95, 0xa1, 0xd0, 0xa8, 0xf7, 0x1a, 0xf5, + 0xdd, 0xa6, 0x96, 0x41, 0x54, 0xaf, 0xd9, 0xe7, 0x3a, 0xed, 0x2a, 0xdb, 0x80, 0x32, 0xa6, 0x76, + 0x9b, 0x4f, 0xea, 0x87, 0xed, 0xbe, 0x96, 0x65, 0xeb, 0x50, 0xea, 0x74, 0xcd, 0x7a, 0xa3, 0xdf, + 0xea, 0x76, 0xb4, 0x9c, 0xfe, 0x43, 0x28, 0x36, 0x8e, 0x9d, 0xe1, 0xc9, 0xab, 0x7a, 0x91, 0x8c, + 0x6e, 0x67, 0x78, 0x22, 0xf4, 0xd3, 0x39, 0xa3, 0xdb, 0x19, 0x9e, 0xe8, 0x4d, 0x28, 0x1d, 0x58, + 0x61, 0xec, 0x52, 0xbd, 0x1e, 0xc3, 0x7a, 0x92, 0xd8, 0x75, 0x46, 0x72, 0xe7, 0x66, 0x89, 0xd6, + 0x9a, 0xa0, 0x8c, 0x34, 0xa1, 0xfe, 0x2e, 0x54, 0x54, 0x00, 0xbb, 0x06, 0x59, 0xdb, 0x19, 0x2d, + 0x91, 0x93, 0x08, 0xd6, 0x9f, 0x43, 0xa5, 0x21, 0x77, 0xa2, 0x57, 0x55, 0xfd, 0x21, 0x54, 0x69, + 0xc5, 0x0f, 0x07, 0x72, 0xc9, 0xaf, 0x2e, 0x59, 0xf2, 0x15, 0xa4, 0x69, 0x0c, 0xc4, 0x9a, 0xff, + 0x08, 0xca, 0x07, 0x61, 0x30, 0x71, 0xc2, 0x98, 0xb2, 0xd5, 0x20, 0x7b, 0xe2, 0x9c, 0x8b, 0x5c, + 0xf1, 0x73, 0xe6, 0x0b, 0x59, 0x55, 0x7d, 0x21, 0x0f, 0xa1, 0x28, 0xd9, 0xbe, 0x36, 0xcf, 0x0f, + 0x50, 0x74, 0x12, 0x8f, 0xeb, 0x44, 0x58, 0xd8, 0x7d, 0x80, 0x49, 0x02, 0x10, 0x1d, 0x27, 0xd5, + 0x7d, 0x91, 0xb9, 0xa1, 0x50, 0xe8, 0x6f, 0x42, 0xe1, 0xb9, 0xeb, 0x9c, 0x8a, 0xe6, 0xbf, 0x74, + 0x9d, 0x53, 0xd9, 0x7c, 0xfc, 0xd6, 0xff, 0xdf, 0x12, 0x14, 0x69, 0x7d, 0xed, 0xbe, 0xda, 0xfd, + 0xf4, 0x4d, 0xb4, 0xa2, 0x6d, 0xb1, 0x9e, 0x72, 0x4b, 0x74, 0x31, 0xbe, 0xba, 0xde, 0x04, 0x50, + 0xd6, 0x3a, 0x97, 0x5c, 0xa5, 0x38, 0x59, 0xe2, 0xa8, 0x4e, 0xd0, 0x5e, 0x14, 0x7d, 0xe9, 0x09, + 0x2b, 0x72, 0x06, 0x60, 0xf7, 0xf9, 0x66, 0x4f, 0x76, 0x23, 0x57, 0x88, 0x2e, 0x48, 0xa5, 0x7e, + 0xe0, 0x39, 0xd2, 0xd4, 0x20, 0x0d, 0x00, 0x13, 0x24, 0xc7, 0x9c, 0x30, 0x42, 0x71, 0x45, 0xfe, + 0x69, 0x43, 0x26, 0xd9, 0xdb, 0x90, 0x43, 0x21, 0x2f, 0x4c, 0x83, 0x0b, 0xb2, 0x07, 0x95, 0x5d, + 0xca, 0x20, 0x02, 0x76, 0x17, 0x0a, 0x24, 0x5a, 0x1c, 0x94, 0x34, 0x4a, 0x6f, 0x4b, 0xa1, 0x6f, + 0x48, 0x34, 0xfb, 0x0e, 0xe4, 0x47, 0x27, 0xce, 0x79, 0x54, 0x5b, 0x27, 0xba, 0x0b, 0x4b, 0xd6, + 0xac, 0xc1, 0x29, 0xd8, 0x6d, 0xa8, 0x86, 0xce, 0xc8, 0x24, 0x87, 0x14, 0x0a, 0x99, 0xa8, 0x56, + 0x25, 0x19, 0x52, 0x09, 0x9d, 0x51, 0x03, 0x81, 0xfd, 0x81, 0x17, 0xb1, 0x3b, 0xb0, 0x46, 0xab, + 0x07, 0x75, 0x21, 0xa5, 0x64, 0xb9, 0x14, 0x0d, 0x81, 0x65, 0x1f, 0x00, 0x08, 0x8d, 0xcb, 0x1c, + 0x9c, 0x93, 0x23, 0x37, 0x59, 0x4c, 0xea, 0xfc, 0x57, 0xf5, 0xb2, 0xb7, 0x21, 0x8f, 0x93, 0x24, + 0xaa, 0x5d, 0xa6, 0x9c, 0x37, 0xd3, 0x33, 0x88, 0x6a, 0x4a, 0x78, 0x76, 0x17, 0x8a, 0x38, 0x51, + 0x4c, 0x1c, 0x8e, 0x9a, 0xaa, 0x82, 0x8a, 0x59, 0x85, 0x3b, 0x83, 0x73, 0xda, 0xfb, 0xd2, 0x63, + 0xf7, 0x20, 0x67, 0xe3, 0x62, 0xbe, 0x42, 0x39, 0x5e, 0x52, 0xc6, 0x05, 0x85, 0xd5, 0xae, 0x33, + 0x22, 0xad, 0x98, 0x68, 0xd8, 0x1e, 0x54, 0x71, 0x1a, 0x3d, 0xa4, 0xcd, 0x1e, 0xbb, 0xaf, 0x76, + 0x95, 0xb8, 0x6e, 0xce, 0x71, 0x75, 0x04, 0x11, 0x75, 0x76, 0xd3, 0x8f, 0xc3, 0x73, 0x63, 0xdd, + 0x57, 0x61, 0xec, 0x2a, 0x9a, 0x2e, 0xed, 0x60, 0x78, 0xe2, 0xd8, 0xb5, 0x37, 0xa4, 0x63, 0x82, + 0xa7, 0xd9, 0xa7, 0xb0, 0x4e, 0x13, 0x0b, 0x93, 0x58, 0x78, 0xed, 0x1a, 0x09, 0x53, 0x75, 0xca, + 0x48, 0x94, 0x91, 0xa6, 0x44, 0x11, 0xef, 0x46, 0x66, 0xec, 0x8c, 0x27, 0x41, 0x88, 0xca, 0xeb, + 0x9b, 0xd2, 0xe1, 0xd2, 0x97, 0x20, 0xdc, 0x88, 0x93, 0x63, 0x27, 0x33, 0x18, 0x8d, 0x22, 0x27, + 0xae, 0x5d, 0xa7, 0x75, 0x53, 0x95, 0xa7, 0x4f, 0x5d, 0x82, 0xd2, 0x46, 0x18, 0x99, 0xf6, 0xb9, + 0x6f, 0x8d, 0xdd, 0x61, 0xed, 0x06, 0xd7, 0x91, 0xdd, 0x68, 0x97, 0x03, 0x54, 0x35, 0x75, 0x3b, + 0xa5, 0xa6, 0x5e, 0x80, 0xbc, 0x3d, 0xc0, 0xe5, 0x78, 0x93, 0xb2, 0xcd, 0xd9, 0x83, 0x96, 0xcd, + 0xde, 0x83, 0xd2, 0x44, 0x8a, 0xc0, 0x9a, 0xae, 0x1a, 0xe3, 0x89, 0x64, 0x34, 0x66, 0x14, 0x68, + 0x1f, 0x3e, 0x71, 0xac, 0x78, 0x1a, 0x3a, 0x4f, 0x3c, 0xeb, 0xa8, 0x76, 0x8b, 0x72, 0x52, 0x41, + 0x57, 0x9f, 0x92, 0xae, 0x4b, 0xad, 0xfe, 0x68, 0x4e, 0xb8, 0xa4, 0x96, 0x86, 0x22, 0x85, 0xf6, + 0x56, 0x54, 0x19, 0xb3, 0x93, 0x27, 0x29, 0x7c, 0xf5, 0x87, 0xc0, 0x16, 0xc7, 0xeb, 0x75, 0x92, + 0x2e, 0x2f, 0x24, 0xdd, 0x77, 0x57, 0x1f, 0x67, 0xf4, 0xe7, 0xb0, 0x9e, 0x5a, 0xc8, 0x4b, 0x25, + 0x36, 0x57, 0x97, 0xac, 0xb1, 0x30, 0x2f, 0x79, 0x42, 0x78, 0xa8, 0x22, 0xd7, 0x3f, 0x12, 0x9e, + 0x2d, 0x9a, 0x08, 0x3d, 0x4a, 0xeb, 0x7f, 0x96, 0x85, 0xca, 0x9e, 0x15, 0x1d, 0xef, 0x5b, 0x93, + 0x5e, 0x6c, 0xc5, 0x11, 0x0e, 0xef, 0xb1, 0x15, 0x1d, 0x8f, 0xad, 0x09, 0x57, 0x5c, 0x33, 0xdc, + 0x6c, 0x16, 0x30, 0xd4, 0x5a, 0x71, 0x62, 0x61, 0xb2, 0xeb, 0x1f, 0x3c, 0x13, 0x36, 0x71, 0x92, + 0x46, 0xb1, 0x12, 0x1d, 0x4f, 0x47, 0xa3, 0xa4, 0x28, 0x99, 0x64, 0xb7, 0x61, 0x5d, 0x7c, 0x92, + 0xd6, 0x7a, 0x26, 0x8e, 0x1d, 0xd3, 0x40, 0xf6, 0x08, 0xca, 0x02, 0xd0, 0x97, 0x42, 0xb0, 0x9a, + 0xf8, 0x3a, 0x66, 0x08, 0x43, 0xa5, 0x62, 0x3f, 0x86, 0x8b, 0x4a, 0xf2, 0x49, 0x10, 0xee, 0x4f, + 0xbd, 0xd8, 0x6d, 0x74, 0x84, 0x8a, 0xf0, 0xc6, 0x02, 0xfb, 0x8c, 0xc4, 0x58, 0xce, 0x99, 0xae, + 0xed, 0xbe, 0xeb, 0x93, 0x4c, 0xcd, 0x1a, 0x69, 0xe0, 0x1c, 0x95, 0x75, 0x46, 0xa2, 0x34, 0x4d, + 0x65, 0x9d, 0xe1, 0x62, 0x13, 0x80, 0x7d, 0x27, 0x3e, 0x0e, 0x6c, 0xd2, 0x0f, 0x93, 0xc5, 0xd6, + 0x53, 0x51, 0x46, 0x9a, 0x12, 0xbb, 0x13, 0x2d, 0xa1, 0xa1, 0x1f, 0x93, 0x96, 0x98, 0x35, 0x64, + 0x12, 0xb7, 0x99, 0xd0, 0xf2, 0x8f, 0x9c, 0xa8, 0x56, 0xde, 0xce, 0xde, 0xcd, 0x18, 0x22, 0xa5, + 0xff, 0xd1, 0x2a, 0xe4, 0xf9, 0x48, 0xbe, 0x01, 0xa5, 0x01, 0x39, 0x8b, 0xd1, 0x32, 0x15, 0x0e, + 0x60, 0x02, 0x74, 0xa6, 0x63, 0xae, 0xdd, 0x09, 0x9f, 0x46, 0xc6, 0xa0, 0x6f, 0xcc, 0x32, 0x98, + 0xc6, 0x58, 0x56, 0x96, 0xa0, 0x22, 0x85, 0x95, 0x08, 0x83, 0x53, 0x9a, 0x0d, 0x39, 0x42, 0xc8, + 0x24, 0xf9, 0x98, 0x69, 0xc7, 0x42, 0xa6, 0x3c, 0xe1, 0x8a, 0x04, 0x68, 0xf8, 0xf1, 0xbc, 0xff, + 0x65, 0x6d, 0xc1, 0xff, 0xc2, 0xae, 0x03, 0xea, 0x8e, 0x43, 0xa7, 0xeb, 0x3b, 0x8d, 0x0e, 0xf5, + 0x70, 0xd1, 0x50, 0x20, 0xb8, 0x40, 0xec, 0x60, 0x42, 0x9d, 0x9a, 0x37, 0xf0, 0x93, 0x7d, 0x9c, + 0xcc, 0x4e, 0x6a, 0xa3, 0xd0, 0xb4, 0x85, 0x44, 0x57, 0xe7, 0xb1, 0x91, 0xa2, 0xc3, 0x9c, 0x50, + 0x4c, 0x73, 0x4d, 0x1b, 0x3f, 0xf5, 0x26, 0x80, 0x11, 0x9c, 0x46, 0x4e, 0x4c, 0x8e, 0xc6, 0xcb, + 0xd4, 0xc4, 0xd4, 0x11, 0x51, 0x70, 0x7a, 0x10, 0x44, 0x89, 0xbd, 0xb9, 0xba, 0xdc, 0xde, 0xd4, + 0x1f, 0x40, 0x01, 0xf7, 0x70, 0x2b, 0xb6, 0xd8, 0x6d, 0xe1, 0xdb, 0xe1, 0x9a, 0x87, 0x70, 0x72, + 0xcd, 0xca, 0x10, 0xde, 0x9e, 0xb6, 0x2c, 0x97, 0x78, 0x6e, 0x2a, 0xe6, 0x5e, 0xb2, 0x7f, 0x88, + 0x0c, 0x85, 0x56, 0xf0, 0x06, 0x94, 0xb0, 0x6a, 0xe4, 0x37, 0x17, 0x72, 0xa1, 0x18, 0x06, 0xa7, + 0x0d, 0x4c, 0xeb, 0xff, 0x3a, 0x03, 0xe5, 0x6e, 0x68, 0xe3, 0xc6, 0xd5, 0x9b, 0x38, 0xc3, 0xd7, + 0x9a, 0xc7, 0xa8, 0x43, 0x04, 0x9e, 0x67, 0x91, 0x88, 0x14, 0xe6, 0x56, 0x02, 0x60, 0x1f, 0x40, + 0x6e, 0x84, 0xa2, 0x30, 0xab, 0x6a, 0xd6, 0x4a, 0xf6, 0xf2, 0x1b, 0x85, 0xa3, 0x41, 0xa4, 0xfa, + 0x6f, 0x24, 0xe5, 0x93, 0x9f, 0x59, 0xf5, 0x2e, 0xaf, 0xd0, 0x39, 0x4f, 0xaf, 0xa1, 0x65, 0x58, + 0x11, 0x72, 0xbb, 0xcd, 0x5e, 0x83, 0xeb, 0xd3, 0xa8, 0x59, 0xf7, 0xcc, 0x27, 0x2d, 0xa3, 0xd7, + 0xd7, 0x72, 0x74, 0x70, 0x44, 0x80, 0x76, 0xbd, 0xd7, 0xd7, 0x8a, 0x0c, 0x60, 0xed, 0xb0, 0xd3, + 0xfa, 0xf1, 0x61, 0x53, 0xd3, 0xf4, 0x7f, 0x96, 0x01, 0x98, 0x39, 0x41, 0xd9, 0x3b, 0x50, 0x3e, + 0xa5, 0x94, 0xa9, 0x78, 0xc7, 0xd5, 0x36, 0x02, 0x47, 0x93, 0x7e, 0xf3, 0x1e, 0x54, 0x12, 0x51, + 0x8f, 0x7b, 0xff, 0xa2, 0x9b, 0xbc, 0x9c, 0xe0, 0x77, 0xce, 0xd9, 0xbb, 0x50, 0x0c, 0xb0, 0x1d, + 0x48, 0x9a, 0x55, 0x37, 0x7e, 0xa5, 0xf9, 0x46, 0x21, 0xe0, 0x09, 0xd4, 0x11, 0x46, 0xa1, 0x34, + 0x9f, 0x13, 0xd2, 0x27, 0x08, 0x6a, 0x78, 0xd6, 0x34, 0x72, 0x0c, 0x8e, 0x4f, 0xa4, 0x74, 0x7e, + 0x26, 0xa5, 0xf5, 0x9f, 0x40, 0xb5, 0x67, 0x8d, 0x27, 0x5c, 0x96, 0x53, 0xc3, 0x18, 0xe4, 0x70, + 0x4e, 0x88, 0xa9, 0x47, 0xdf, 0xb8, 0xe8, 0x0e, 0x9c, 0x70, 0xe8, 0xf8, 0x72, 0x8d, 0xca, 0x24, + 0x8a, 0xdf, 0x43, 0x94, 0xe6, 0x46, 0x70, 0x2a, 0xc5, 0xb9, 0x4c, 0xeb, 0x7f, 0x23, 0x03, 0x65, + 0xa5, 0x1a, 0xec, 0x01, 0xe4, 0x48, 0x99, 0xcc, 0xa8, 0x82, 0x50, 0x21, 0xe0, 0xdf, 0x5c, 0xfd, + 0x40, 0x42, 0x76, 0x07, 0xf2, 0x51, 0x6c, 0x85, 0xd2, 0x9f, 0xae, 0x29, 0x1c, 0x3b, 0xc1, 0xd4, + 0xb7, 0x0d, 0x8e, 0x66, 0x3a, 0x64, 0x1d, 0xdf, 0x16, 0x0e, 0x87, 0x45, 0x2a, 0x44, 0xea, 0xdb, + 0x50, 0x4a, 0xb2, 0xc7, 0x29, 0x60, 0x74, 0x5f, 0xf4, 0xb4, 0x15, 0x56, 0x82, 0xbc, 0x51, 0xef, + 0x3c, 0x6d, 0x6a, 0x19, 0xfd, 0x8f, 0x33, 0x00, 0x33, 0x2e, 0x76, 0x3f, 0x55, 0xdb, 0xab, 0xf3, + 0xb9, 0xde, 0xa7, 0xbf, 0x4a, 0x65, 0xaf, 0x41, 0x69, 0xea, 0x13, 0xd0, 0xb1, 0xc5, 0x4e, 0x34, + 0x03, 0xa0, 0x05, 0x24, 0xa3, 0x3f, 0xe6, 0x2c, 0xa0, 0x97, 0x96, 0xa7, 0x7f, 0x17, 0x4a, 0x49, + 0x76, 0x68, 0xd4, 0x3d, 0xe9, 0xb6, 0xdb, 0xdd, 0x17, 0xad, 0xce, 0x53, 0x6d, 0x05, 0x93, 0x07, + 0x46, 0xb3, 0xd1, 0xdc, 0xc5, 0x64, 0x06, 0xe7, 0x6c, 0xe3, 0xd0, 0x30, 0x9a, 0x9d, 0xbe, 0x69, + 0x74, 0x5f, 0x68, 0xab, 0xfa, 0xff, 0x95, 0x83, 0xcd, 0xae, 0xbf, 0x3b, 0x9d, 0x78, 0xee, 0xd0, + 0x8a, 0x9d, 0x67, 0xce, 0x79, 0x23, 0x3e, 0xc3, 0xdd, 0xd7, 0x8a, 0xe3, 0x90, 0x2f, 0xe6, 0x92, + 0xc1, 0x13, 0xdc, 0x29, 0x11, 0x39, 0x61, 0x4c, 0x3e, 0x17, 0x75, 0x15, 0x57, 0x39, 0xbc, 0x11, + 0x78, 0xb4, 0x96, 0xd9, 0xf7, 0xe1, 0x22, 0x77, 0x64, 0x70, 0x4a, 0x54, 0x60, 0x4d, 0x5a, 0xcc, + 0xd9, 0x85, 0xa9, 0xcb, 0x38, 0x21, 0xb2, 0x22, 0x19, 0x89, 0xb0, 0x1b, 0x50, 0x9e, 0xb1, 0xcb, + 0x43, 0x2a, 0x48, 0x08, 0xa9, 0x26, 0x68, 0x78, 0xcb, 0x5a, 0x9b, 0xae, 0x7d, 0x46, 0x2e, 0x9e, + 0xbc, 0x51, 0x0d, 0x66, 0x8d, 0xc1, 0x4d, 0xf8, 0x33, 0xd8, 0x4c, 0x51, 0x52, 0x2d, 0xd6, 0xa8, + 0x16, 0xef, 0x4a, 0x07, 0xe9, 0x5c, 0xeb, 0x55, 0x08, 0x56, 0x87, 0x6b, 0xa4, 0x1b, 0x41, 0x1a, + 0x2a, 0x74, 0x11, 0xf7, 0xc8, 0x0f, 0x42, 0x47, 0x08, 0xfc, 0xa2, 0x1b, 0xb5, 0x28, 0x3d, 0xb3, + 0x7f, 0x94, 0x43, 0x55, 0xbe, 0xbf, 0xc8, 0xf3, 0x42, 0x8e, 0x76, 0xf9, 0x0e, 0x9a, 0x33, 0x0a, + 0x94, 0x6e, 0xd9, 0xec, 0x96, 0x50, 0x67, 0x4d, 0x69, 0xd2, 0x00, 0x99, 0x34, 0x15, 0x02, 0x3e, + 0xe7, 0xb0, 0xab, 0x1d, 0xd8, 0x5a, 0x56, 0xc9, 0x25, 0x6a, 0xd8, 0xb6, 0xaa, 0x86, 0xcd, 0x19, + 0xed, 0x33, 0x95, 0xec, 0x6f, 0x65, 0xa0, 0xb2, 0xeb, 0xd8, 0xd3, 0xc9, 0x8f, 0x02, 0xd7, 0xc7, + 0x09, 0xf0, 0x21, 0x54, 0x02, 0xcf, 0xa6, 0xd1, 0x53, 0x62, 0x03, 0x52, 0x27, 0x46, 0xc2, 0xb9, + 0x0d, 0x81, 0x67, 0x37, 0x02, 0x8f, 0x22, 0x09, 0xde, 0x83, 0x0b, 0xdc, 0xa1, 0x21, 0xfc, 0x7b, + 0x67, 0x9c, 0x79, 0x95, 0x46, 0x46, 0xe3, 0x28, 0xae, 0x1c, 0x11, 0xf9, 0xaf, 0xc1, 0x96, 0x42, + 0x8e, 0x23, 0xc3, 0xe9, 0x17, 0x27, 0xc9, 0x66, 0xc2, 0x2b, 0x8f, 0x6c, 0xf4, 0xdf, 0x59, 0x85, + 0x12, 0x77, 0x87, 0x60, 0x7d, 0xef, 0x42, 0x21, 0x18, 0x7c, 0x61, 0x86, 0x89, 0x9b, 0x60, 0xe1, + 0xa4, 0x71, 0x2d, 0x18, 0x7c, 0x61, 0x38, 0x23, 0xf6, 0x8e, 0xdc, 0xe7, 0x6d, 0x67, 0x24, 0x3a, + 0xa5, 0x9a, 0xb6, 0x47, 0xc4, 0xbe, 0x8f, 0xb6, 0xf2, 0x23, 0x28, 0xcf, 0x66, 0x7c, 0x54, 0x2b, + 0xbc, 0xba, 0x17, 0x92, 0x05, 0x10, 0x21, 0x13, 0x77, 0x09, 0x71, 0xa6, 0xe2, 0xab, 0x99, 0x38, + 0x19, 0x31, 0x7d, 0x0a, 0xd5, 0x99, 0x8c, 0x27, 0xbe, 0xd2, 0x2b, 0xf9, 0xd6, 0x13, 0x4a, 0x3a, + 0xf1, 0xf8, 0xdb, 0x19, 0x28, 0xb5, 0x78, 0xf1, 0xf1, 0x19, 0xbb, 0x09, 0xd9, 0xaf, 0xe8, 0x05, + 0xc4, 0xb1, 0x7b, 0xb0, 0x69, 0xd9, 0xb6, 0x69, 0x8d, 0x46, 0xce, 0x30, 0x76, 0x6c, 0x13, 0x55, + 0x20, 0x21, 0x73, 0x36, 0x2c, 0xdb, 0xae, 0x0b, 0x38, 0xc9, 0x6e, 0x5c, 0xf3, 0x91, 0x29, 0x0d, + 0xcf, 0xd9, 0x91, 0x72, 0xd1, 0xa8, 0xba, 0x91, 0xb0, 0x3b, 0xb9, 0x2f, 0x39, 0xd5, 0xb1, 0xb9, + 0xaf, 0xee, 0x58, 0xfd, 0xf7, 0x56, 0x01, 0x0c, 0x67, 0xe2, 0x59, 0x43, 0xe7, 0x7f, 0x9b, 0x4a, + 0xa3, 0x58, 0x4a, 0x06, 0xd6, 0xb7, 0x65, 0x08, 0x86, 0x1c, 0x44, 0xdf, 0x66, 0x3f, 0x84, 0x37, + 0x43, 0xe7, 0x34, 0x74, 0x63, 0xc7, 0x1c, 0x85, 0xc1, 0xd8, 0x4c, 0x49, 0x1e, 0x5c, 0x98, 0x25, + 0xaa, 0xc4, 0x15, 0x41, 0xf4, 0x24, 0x0c, 0xc6, 0x69, 0xe9, 0xa3, 0xff, 0xc7, 0x22, 0x94, 0xeb, + 0xbe, 0xe5, 0x9d, 0xff, 0xd4, 0xa1, 0x98, 0x00, 0xf2, 0xae, 0x4e, 0xa6, 0x31, 0x6f, 0x2e, 0x3f, + 0x30, 0x2b, 0x11, 0x84, 0x1a, 0x7a, 0x03, 0xca, 0xc1, 0x34, 0x4e, 0xf0, 0xfc, 0x08, 0x0d, 0x38, + 0x88, 0x08, 0x12, 0xfe, 0xc4, 0x73, 0x2f, 0xf9, 0xc9, 0xfc, 0x99, 0xf1, 0x27, 0x2a, 0x71, 0xc2, + 0x4f, 0x04, 0x28, 0x8d, 0xdc, 0x31, 0x35, 0x38, 0x9a, 0x8e, 0x1d, 0xde, 0xe8, 0x2c, 0x8f, 0x4f, + 0x6b, 0x08, 0x18, 0xe6, 0x32, 0x76, 0xc6, 0x41, 0x78, 0xce, 0x73, 0x59, 0xe3, 0xb9, 0x70, 0x10, + 0xe5, 0xf2, 0x2e, 0xb0, 0x53, 0xcb, 0x8d, 0xcd, 0x74, 0x56, 0xdc, 0x0c, 0xd1, 0x10, 0xd3, 0x57, + 0xb3, 0xbb, 0x04, 0x6b, 0xb6, 0x1b, 0x9d, 0xb4, 0xba, 0xc2, 0x04, 0x11, 0x29, 0x6c, 0x4b, 0x34, + 0xb4, 0x50, 0x03, 0x8a, 0x1d, 0xae, 0x2e, 0x67, 0x8d, 0x12, 0x42, 0x76, 0x10, 0x80, 0x3b, 0xa8, + 0xef, 0xc4, 0xa7, 0x41, 0x88, 0x9c, 0xdc, 0xc2, 0x98, 0x01, 0x50, 0xd3, 0x40, 0x52, 0x2c, 0x88, + 0xbc, 0x41, 0x59, 0x23, 0x49, 0xa3, 0xee, 0xce, 0x97, 0x2f, 0x61, 0x2b, 0xbc, 0xfa, 0x33, 0x08, + 0xbb, 0x0d, 0x55, 0xaa, 0x3e, 0x59, 0x20, 0xd8, 0x06, 0x3a, 0xe5, 0xca, 0x1a, 0x15, 0x84, 0x92, + 0x33, 0x01, 0xa9, 0x3e, 0x85, 0x2b, 0xa9, 0xf6, 0x99, 0x56, 0x18, 0x5a, 0xe7, 0xe6, 0xd8, 0xfa, + 0x22, 0x08, 0xc9, 0xf1, 0x93, 0x35, 0x2e, 0xa9, 0xdd, 0x56, 0x47, 0xf4, 0x3e, 0x62, 0x5f, 0xc9, + 0xea, 0xfa, 0x41, 0x48, 0x5e, 0xa1, 0xa5, 0xac, 0x88, 0x25, 0x17, 0x06, 0x0d, 0x30, 0x99, 0x43, + 0x11, 0x8f, 0x6b, 0x34, 0xca, 0x04, 0xdb, 0x21, 0x10, 0x1a, 0x04, 0xd1, 0x23, 0x2e, 0x59, 0x37, + 0x45, 0x90, 0xd1, 0x23, 0x92, 0xbf, 0x1c, 0x71, 0xec, 0x58, 0x36, 0x9d, 0x9c, 0x11, 0x62, 0xcf, + 0xb1, 0xe8, 0x5c, 0x3a, 0x7a, 0x64, 0x4e, 0xa6, 0x31, 0x0f, 0x48, 0x34, 0xf2, 0xd1, 0xa3, 0x83, + 0x69, 0x2c, 0xc0, 0x47, 0x4e, 0x4c, 0x61, 0x88, 0x04, 0x7e, 0xea, 0xc4, 0xb8, 0x11, 0x46, 0x8f, + 0xa4, 0x17, 0xfc, 0xa2, 0xe8, 0xdb, 0x47, 0xc2, 0xcd, 0xad, 0xc3, 0x7a, 0x82, 0x34, 0xc7, 0x53, + 0x1e, 0x81, 0x98, 0x35, 0xca, 0x92, 0x60, 0x7f, 0xea, 0xe1, 0xc0, 0x0e, 0xad, 0xe1, 0xb1, 0x63, + 0x86, 0x58, 0x95, 0xcb, 0x7c, 0xe8, 0x08, 0x62, 0x60, 0x6d, 0xde, 0x00, 0x9e, 0x30, 0x8f, 0xdd, + 0x98, 0xbc, 0x53, 0x59, 0xa3, 0x48, 0x80, 0x3d, 0x37, 0x46, 0xb1, 0xc0, 0x91, 0x62, 0x06, 0x52, + 0x16, 0x57, 0x88, 0x68, 0x83, 0x10, 0xfb, 0x04, 0xa7, 0x8c, 0xee, 0x82, 0x96, 0xa2, 0xc5, 0xfc, + 0xae, 0x12, 0x69, 0x55, 0x21, 0xc5, 0x5c, 0xef, 0x00, 0x67, 0x36, 0x71, 0xea, 0xf1, 0x3c, 0xdf, + 0xe0, 0xe6, 0x30, 0x81, 0x77, 0xdd, 0xe8, 0x84, 0x72, 0xbc, 0x0d, 0x55, 0x85, 0x0e, 0xf3, 0xbb, + 0xc6, 0x67, 0x46, 0x42, 0x96, 0xaa, 0x63, 0xe8, 0x8c, 0x83, 0x58, 0x34, 0xf3, 0x4d, 0xa5, 0x8e, + 0x06, 0xc1, 0xd3, 0x75, 0x14, 0xb4, 0x98, 0xe7, 0x75, 0xa5, 0x8e, 0x9c, 0x14, 0x73, 0xbd, 0x09, + 0x15, 0x94, 0x22, 0xb1, 0xe3, 0xf3, 0xc5, 0x7f, 0x83, 0x77, 0xac, 0x80, 0xd1, 0xea, 0xbf, 0x09, + 0x15, 0xde, 0xf3, 0x42, 0x5c, 0x6e, 0x73, 0x12, 0x01, 0x43, 0x12, 0xfd, 0x67, 0x19, 0xb8, 0xda, + 0xa5, 0x03, 0x45, 0x12, 0x78, 0xfb, 0x4e, 0x14, 0x59, 0x47, 0xce, 0x93, 0x20, 0x7c, 0x32, 0xfd, + 0xe9, 0x4f, 0xcf, 0xd9, 0x5d, 0xd8, 0x38, 0xb0, 0x42, 0xc7, 0x8f, 0x93, 0x03, 0x27, 0xa1, 0x5c, + 0xcc, 0x83, 0xd9, 0x63, 0xd0, 0x38, 0xe8, 0x30, 0x51, 0xd3, 0x84, 0xa1, 0x92, 0xf6, 0x0f, 0x2f, + 0x50, 0xe9, 0xff, 0x7d, 0x1b, 0x72, 0x9d, 0xc0, 0x76, 0xd8, 0xfb, 0x50, 0xa2, 0xf0, 0x3f, 0x45, + 0x71, 0x16, 0x8e, 0x05, 0x44, 0xd3, 0x1f, 0xd2, 0x98, 0x8b, 0xbe, 0xf8, 0x7a, 0x75, 0xc0, 0xe0, + 0x4d, 0xd2, 0xfd, 0xe9, 0xb4, 0x0d, 0x65, 0x7a, 0x59, 0xf8, 0x27, 0xc8, 0x9c, 0xe6, 0x18, 0x94, + 0x08, 0xe4, 0x99, 0x0d, 0x1d, 0x9f, 0x34, 0xcc, 0xbc, 0x91, 0xa4, 0xc9, 0xe2, 0x0a, 0x03, 0xdc, + 0x7f, 0xf8, 0xba, 0xc9, 0x2f, 0xb1, 0xb8, 0x38, 0x9e, 0x16, 0xd2, 0xfb, 0x50, 0xfa, 0x22, 0x70, + 0x7d, 0x5e, 0xf1, 0xb5, 0x85, 0x8a, 0xa3, 0x52, 0xc5, 0x2b, 0xfe, 0x85, 0xf8, 0x62, 0xb7, 0xa0, + 0x10, 0xf8, 0x3c, 0xef, 0xc2, 0x42, 0xde, 0x6b, 0x81, 0xdf, 0xe6, 0xd1, 0x28, 0xeb, 0x6e, 0x64, + 0x86, 0xee, 0xd1, 0x71, 0x6c, 0x22, 0xa7, 0x38, 0xa5, 0x2b, 0xbb, 0x91, 0x81, 0x30, 0xcc, 0x16, + 0x0d, 0xc9, 0x91, 0xeb, 0xe1, 0x36, 0x47, 0x99, 0x95, 0x16, 0x32, 0x03, 0x8e, 0xa6, 0x0c, 0xdf, + 0x82, 0xe2, 0x51, 0x18, 0x4c, 0x27, 0x68, 0x19, 0xc2, 0x02, 0x65, 0x81, 0x70, 0x3b, 0xe7, 0x28, + 0xf4, 0xe9, 0xd3, 0xf5, 0x8f, 0x4c, 0x32, 0xa2, 0xcb, 0xdb, 0xd9, 0xbb, 0x45, 0xa3, 0x22, 0x81, + 0x64, 0x1e, 0xbf, 0x05, 0x45, 0xeb, 0xe8, 0xc8, 0x14, 0x41, 0x35, 0x0b, 0x79, 0x59, 0x47, 0x47, + 0x54, 0xe4, 0x7d, 0x58, 0x3f, 0x75, 0x7d, 0x33, 0x9a, 0x38, 0x43, 0x4e, 0xbb, 0xbe, 0xd8, 0x95, + 0xa7, 0xae, 0x8f, 0xb6, 0x23, 0xd1, 0xab, 0xc6, 0x6b, 0xf5, 0xb5, 0xc6, 0xeb, 0x36, 0xe4, 0x3d, + 0x77, 0xec, 0xc6, 0x22, 0xcc, 0x26, 0xa5, 0xdd, 0x12, 0x82, 0xe9, 0xb0, 0x26, 0xbc, 0xb6, 0xda, + 0x02, 0x89, 0xc0, 0xa4, 0x95, 0x80, 0xcd, 0xd7, 0x28, 0x01, 0x8a, 0xa6, 0xc9, 0xbe, 0x5a, 0xd3, + 0xfc, 0x88, 0x54, 0x3a, 0xc7, 0x8f, 0x4d, 0xc9, 0x70, 0x61, 0x39, 0x43, 0x85, 0x93, 0x75, 0x39, + 0xdb, 0x07, 0x50, 0x0e, 0xc9, 0xab, 0x62, 0x92, 0x0b, 0x66, 0x4b, 0x35, 0x4b, 0x67, 0xee, 0x16, + 0x03, 0xc2, 0x99, 0xeb, 0xa5, 0x0e, 0x1b, 0xb3, 0xb0, 0x41, 0x1e, 0x7f, 0x79, 0x51, 0x75, 0xeb, + 0xa6, 0xe2, 0x0c, 0xa5, 0x12, 0xe9, 0xa6, 0x82, 0x0f, 0x6f, 0xc1, 0x3a, 0x0f, 0x18, 0xe0, 0xc7, + 0xba, 0x11, 0xc9, 0xe9, 0x92, 0x51, 0x21, 0x20, 0x3f, 0xf2, 0x8d, 0xd8, 0x7d, 0x00, 0xa9, 0x00, + 0xc5, 0x67, 0x24, 0xa8, 0x93, 0xd6, 0x70, 0x69, 0xde, 0x88, 0xcf, 0x8c, 0x92, 0x2d, 0x3f, 0x51, + 0xfe, 0x0c, 0x5c, 0xdf, 0xc6, 0x79, 0x14, 0x5b, 0x47, 0x51, 0xad, 0x46, 0xcb, 0xac, 0x2c, 0x60, + 0x7d, 0xeb, 0x28, 0x42, 0x43, 0xc3, 0xe2, 0xfa, 0x0e, 0xaf, 0xf7, 0x15, 0xd5, 0x0b, 0xa1, 0x68, + 0x42, 0x46, 0xd9, 0x52, 0xd4, 0xa2, 0x4f, 0x80, 0xc9, 0x13, 0x22, 0xc5, 0x6e, 0xb8, 0xba, 0x30, + 0xb5, 0x36, 0xc4, 0x11, 0x51, 0x12, 0xeb, 0xfc, 0x09, 0xac, 0xa7, 0xd5, 0xc2, 0x6b, 0x4b, 0xce, + 0x51, 0x68, 0xd4, 0x8d, 0xca, 0x50, 0x55, 0x14, 0x6f, 0xf1, 0x88, 0x55, 0x92, 0xc1, 0xc4, 0xc8, + 0xcf, 0x0a, 0x2a, 0x7e, 0x10, 0x37, 0x24, 0x0c, 0xfb, 0x47, 0x9a, 0x0b, 0xf1, 0x19, 0x89, 0xed, + 0xa4, 0x7f, 0x12, 0x05, 0x1d, 0xd5, 0x2f, 0xa9, 0xab, 0xe3, 0x50, 0x73, 0x25, 0x98, 0x18, 0x6e, + 0xa4, 0x86, 0x3a, 0xd1, 0x8e, 0x0d, 0x08, 0x67, 0x9a, 0xf2, 0x0d, 0x28, 0x47, 0xc1, 0x34, 0x1c, + 0x3a, 0x66, 0x14, 0x3b, 0x93, 0xda, 0x36, 0xf5, 0x28, 0x70, 0x50, 0x2f, 0x76, 0x26, 0xec, 0x31, + 0x54, 0x27, 0x21, 0x76, 0x66, 0x52, 0x0f, 0x5d, 0x6d, 0xe2, 0x41, 0xe8, 0xcc, 0xaa, 0x52, 0x99, + 0x28, 0x29, 0xf6, 0x03, 0xd8, 0x54, 0x38, 0xa7, 0x27, 0xc4, 0x7c, 0x8b, 0x98, 0xb7, 0xe6, 0x98, + 0x0f, 0x4f, 0x90, 0xbd, 0x3a, 0x49, 0xa5, 0x59, 0x7d, 0xce, 0x2a, 0x47, 0x8d, 0xf7, 0x36, 0xf1, + 0x5f, 0x7e, 0x85, 0xa9, 0x9d, 0x32, 0xd7, 0x9f, 0xf1, 0x53, 0x83, 0x56, 0xd4, 0xf4, 0xed, 0xda, + 0x5b, 0xfc, 0x5a, 0x01, 0x25, 0xd8, 0x23, 0xa8, 0x70, 0xdd, 0x8b, 0x82, 0xfa, 0xa2, 0xda, 0x1d, + 0xd5, 0x2d, 0x49, 0x0a, 0x18, 0x21, 0x8c, 0xb2, 0x97, 0x7c, 0x47, 0xec, 0x63, 0xd8, 0xe4, 0x3e, + 0x63, 0x55, 0x3e, 0xbe, 0xbd, 0x38, 0x45, 0x88, 0xe8, 0xc9, 0x4c, 0x48, 0x1a, 0x70, 0x25, 0x9c, + 0xfa, 0xa4, 0x8f, 0x09, 0xce, 0x49, 0x18, 0x0c, 0x1c, 0xce, 0x7f, 0x97, 0xf8, 0x45, 0x73, 0x0c, + 0x4e, 0xc6, 0x79, 0x49, 0x30, 0x5d, 0x0a, 0x55, 0xd0, 0x01, 0xf2, 0xbd, 0x22, 0xcf, 0xc1, 0xd4, + 0xf5, 0x6c, 0x9e, 0xe7, 0x77, 0xbe, 0x49, 0x9e, 0x3b, 0xc8, 0x47, 0x79, 0x32, 0xc8, 0x4d, 0xa7, + 0xae, 0x5d, 0xbb, 0xc7, 0xe3, 0xef, 0xf0, 0x9b, 0xbd, 0x05, 0xd5, 0xd0, 0x19, 0x4e, 0xc3, 0xc8, + 0x7d, 0xe9, 0x98, 0x91, 0xeb, 0x9f, 0xd4, 0xde, 0xa1, 0x7e, 0x5c, 0x4f, 0xa0, 0x3d, 0xd7, 0x3f, + 0xc1, 0x79, 0xe7, 0x9c, 0xc5, 0x4e, 0xe8, 0xf3, 0x38, 0xe3, 0x77, 0xd5, 0x79, 0xd7, 0x24, 0x04, + 0xca, 0x05, 0x03, 0x9c, 0xe4, 0x7b, 0x6e, 0x72, 0x44, 0x7c, 0x72, 0xdc, 0xff, 0x5a, 0x93, 0xa3, + 0x47, 0x93, 0xe3, 0x0e, 0x14, 0x5d, 0x3f, 0x76, 0xc2, 0x97, 0x96, 0x57, 0x7b, 0xb0, 0x20, 0x8a, + 0x13, 0x1c, 0xbb, 0x0d, 0x85, 0xc8, 0x73, 0x51, 0x3e, 0xd4, 0xde, 0x5f, 0x20, 0x93, 0x28, 0x76, + 0x17, 0x4a, 0xc9, 0x15, 0x99, 0xda, 0x07, 0x0b, 0x74, 0x33, 0x24, 0xbb, 0x0e, 0xb9, 0x53, 0x9c, + 0x50, 0x0f, 0x17, 0xdd, 0xc8, 0x08, 0xc7, 0xbd, 0x7b, 0xe4, 0x7a, 0x1e, 0xdf, 0xbb, 0x1f, 0x2d, + 0xec, 0xdd, 0x4f, 0x5c, 0xcf, 0xe3, 0x7b, 0xf7, 0x48, 0x7c, 0xe1, 0xce, 0x47, 0x1c, 0xd8, 0x92, + 0x0f, 0x17, 0x77, 0x3e, 0xc4, 0x3d, 0xa7, 0xcb, 0x44, 0xe5, 0x88, 0x7c, 0xa3, 0xdc, 0xc5, 0xfb, + 0x91, 0xda, 0x57, 0x69, 0xa7, 0xa9, 0x01, 0x51, 0x92, 0x46, 0x65, 0x58, 0x78, 0x86, 0x5d, 0xfb, + 0xac, 0xf6, 0x31, 0x8f, 0x52, 0xe7, 0x90, 0x96, 0x7d, 0xc6, 0xde, 0x87, 0x75, 0x19, 0x06, 0x82, + 0xc5, 0x45, 0xb5, 0x4f, 0x16, 0x6a, 0x90, 0x26, 0x60, 0xbb, 0x50, 0x19, 0xa1, 0x2e, 0x37, 0xe6, + 0xaa, 0x5d, 0xed, 0x31, 0x55, 0x64, 0x5b, 0xee, 0xaa, 0xaf, 0x52, 0xfd, 0x8c, 0x14, 0x17, 0xbb, + 0x0f, 0xcc, 0x1d, 0xf1, 0xf1, 0x44, 0xb3, 0x95, 0xab, 0x6f, 0xb5, 0x4f, 0x69, 0x76, 0x2d, 0xc1, + 0xd0, 0x41, 0x91, 0xe3, 0xdb, 0xe6, 0x38, 0x12, 0x3a, 0xc2, 0x77, 0xa9, 0x9e, 0x42, 0x1a, 0x26, + 0x57, 0xe9, 0xc4, 0x66, 0x54, 0x46, 0xda, 0xfd, 0x88, 0xab, 0x0c, 0x9f, 0x02, 0x4e, 0xd7, 0x97, + 0x33, 0xd6, 0x5f, 0xfb, 0x4a, 0x56, 0xa4, 0x95, 0xac, 0x8f, 0xa1, 0x6a, 0x3b, 0xf6, 0x74, 0x42, + 0xea, 0x12, 0x4d, 0xd1, 0xef, 0xa9, 0xc2, 0x4f, 0x75, 0x71, 0x19, 0x15, 0x5b, 0x75, 0x78, 0x7d, + 0x02, 0x1b, 0xd2, 0x17, 0x15, 0x0b, 0xb7, 0xd5, 0xf7, 0xd5, 0x62, 0x13, 0x57, 0x93, 0xb1, 0x3e, + 0x95, 0x9f, 0x54, 0xe4, 0x23, 0x58, 0xa7, 0x5d, 0x37, 0xf2, 0xad, 0x49, 0x74, 0x1c, 0xc4, 0xb5, + 0x5f, 0x57, 0x15, 0x88, 0x9e, 0x80, 0x1a, 0x15, 0x24, 0x92, 0x29, 0xdc, 0x4d, 0x66, 0xeb, 0x74, + 0x18, 0x3b, 0xb5, 0x1f, 0xf0, 0xdd, 0x24, 0x01, 0x36, 0x62, 0x87, 0x3d, 0x02, 0xb0, 0x26, 0x13, + 0xef, 0x9c, 0x4f, 0xcd, 0x1f, 0xd2, 0xd4, 0xdc, 0x52, 0xa6, 0x66, 0x1d, 0x91, 0x34, 0x37, 0x4b, + 0x96, 0xfc, 0x64, 0x0f, 0xa1, 0x32, 0x09, 0xa2, 0xd8, 0xb4, 0xc7, 0x1e, 0xb5, 0xbf, 0xae, 0xae, + 0xed, 0x83, 0x20, 0x8a, 0x77, 0xc7, 0x1e, 0xed, 0x29, 0x93, 0xe4, 0x9b, 0xb5, 0xe1, 0x42, 0x4a, + 0x6e, 0x5b, 0x74, 0x2c, 0x5b, 0xdb, 0xa1, 0x12, 0xaf, 0x29, 0x25, 0x2a, 0xf2, 0x5b, 0xc4, 0x24, + 0x6d, 0x06, 0xf3, 0x20, 0xb4, 0x89, 0xf8, 0x18, 0x24, 0x81, 0x79, 0x0d, 0xae, 0x4a, 0x10, 0x54, + 0x46, 0xe6, 0x3d, 0x86, 0x8d, 0x19, 0x15, 0x36, 0x30, 0xaa, 0xed, 0xaa, 0x33, 0x59, 0x09, 0x9f, + 0x5d, 0x97, 0x8c, 0x08, 0x8b, 0xa8, 0xef, 0x02, 0xcf, 0x9b, 0x4e, 0x84, 0x28, 0xad, 0x35, 0x45, + 0xdf, 0x11, 0x90, 0x4b, 0x49, 0xfd, 0x1f, 0xe7, 0xa1, 0x28, 0xed, 0x05, 0x56, 0x86, 0xc2, 0x61, + 0xe7, 0x59, 0xa7, 0xfb, 0xa2, 0xc3, 0x6f, 0xfa, 0xd4, 0x7b, 0xbd, 0xa6, 0xd1, 0xd7, 0x6c, 0x56, + 0x05, 0xa0, 0x58, 0x7e, 0xb3, 0xd7, 0xa8, 0x77, 0xf8, 0xcd, 0x1f, 0xba, 0x41, 0xc0, 0xd3, 0xab, + 0x6c, 0x13, 0xd6, 0x9f, 0x1c, 0x76, 0x28, 0x3e, 0x8a, 0x83, 0xb2, 0x08, 0x6a, 0x7e, 0xc6, 0x4f, + 0x85, 0x38, 0x28, 0x87, 0xa0, 0xfd, 0x7a, 0xbf, 0x69, 0xb4, 0x24, 0x28, 0x4f, 0xa1, 0x56, 0xdd, + 0x43, 0xa3, 0x21, 0x72, 0x5a, 0x63, 0x17, 0x61, 0x33, 0x61, 0x93, 0x59, 0x6a, 0x05, 0xac, 0xd9, + 0x81, 0xd1, 0xfd, 0x51, 0xb3, 0xd1, 0xd7, 0x80, 0x8e, 0x98, 0x9e, 0x3e, 0xd5, 0xca, 0xac, 0x02, + 0xc5, 0xdd, 0x56, 0xaf, 0xdf, 0xea, 0x34, 0xfa, 0x5a, 0x05, 0x2b, 0xfc, 0xa4, 0xd5, 0xee, 0x37, + 0x0d, 0x6d, 0x9d, 0x15, 0x21, 0xf7, 0xa3, 0x6e, 0xab, 0xa3, 0x55, 0xe9, 0x4e, 0x43, 0x7d, 0xff, + 0xa0, 0xdd, 0xd4, 0x36, 0x10, 0xda, 0xeb, 0x1a, 0x7d, 0x4d, 0x43, 0xe8, 0x8b, 0x56, 0x67, 0xb7, + 0xfb, 0x42, 0xdb, 0x64, 0x25, 0xc8, 0x1f, 0x76, 0xb0, 0x18, 0xc6, 0xd6, 0xa1, 0x44, 0x9f, 0x66, + 0xbd, 0xdd, 0xd6, 0x2e, 0x28, 0xe7, 0x52, 0x5b, 0x88, 0xa2, 0x53, 0xae, 0x1e, 0xd6, 0xe1, 0x22, + 0xb6, 0x25, 0x49, 0x12, 0xf5, 0x25, 0xcc, 0x67, 0xbf, 0xd5, 0x39, 0xec, 0x69, 0x97, 0x91, 0x98, + 0x3e, 0x09, 0x53, 0xc3, 0x7c, 0x5a, 0x1d, 0xea, 0xca, 0xeb, 0xf8, 0xbd, 0xdb, 0x6c, 0x37, 0xfb, + 0x4d, 0xed, 0x06, 0xb6, 0xca, 0x68, 0x1e, 0xb4, 0xeb, 0x8d, 0xa6, 0xb6, 0x8d, 0x89, 0x76, 0xb7, + 0xf1, 0xcc, 0xec, 0x1e, 0x68, 0x37, 0xd9, 0x16, 0x68, 0xdd, 0x8e, 0xb9, 0x7b, 0x78, 0xd0, 0x6e, + 0x35, 0xea, 0xfd, 0xa6, 0xf9, 0xac, 0xf9, 0xb9, 0xa6, 0x63, 0xb7, 0x1f, 0x18, 0x4d, 0x53, 0xe4, + 0x75, 0x8b, 0x69, 0x50, 0x79, 0x72, 0xf8, 0x93, 0x9f, 0x7c, 0x6e, 0x8a, 0x76, 0xbf, 0x85, 0xd5, + 0x9a, 0x51, 0x98, 0x87, 0xcf, 0xb4, 0x3b, 0x73, 0xa0, 0xde, 0x33, 0xed, 0x6d, 0xec, 0x37, 0x39, + 0x10, 0xda, 0x5d, 0x24, 0x30, 0x9a, 0x8d, 0x43, 0xa3, 0xd7, 0x7a, 0xde, 0x34, 0x1b, 0xfd, 0xa6, + 0xf6, 0x1d, 0xea, 0xa8, 0x56, 0xe7, 0x99, 0x76, 0x0f, 0x5b, 0x82, 0x5f, 0x7c, 0x78, 0xde, 0x61, + 0x0c, 0xaa, 0x33, 0x5a, 0x82, 0xbd, 0x8b, 0x24, 0x3b, 0x46, 0xb7, 0xbe, 0xdb, 0xa8, 0xf7, 0xfa, + 0xda, 0x7b, 0xd8, 0x0d, 0xbd, 0x83, 0x76, 0xab, 0xaf, 0xdd, 0xc7, 0xb6, 0x3e, 0xad, 0xf7, 0xf7, + 0x9a, 0x86, 0xf6, 0x00, 0x47, 0xba, 0xdf, 0xda, 0x6f, 0x9a, 0xa2, 0xdb, 0x1f, 0x62, 0x19, 0x4f, + 0x5a, 0xed, 0xb6, 0xf6, 0x88, 0x8e, 0x5e, 0xea, 0x46, 0xbf, 0x45, 0x63, 0xfd, 0x21, 0x66, 0x50, + 0x3f, 0x38, 0x68, 0x7f, 0xae, 0x7d, 0x84, 0x0d, 0xdc, 0x3f, 0x6c, 0xf7, 0x5b, 0xe6, 0xe1, 0xc1, + 0x6e, 0xbd, 0xdf, 0xd4, 0x3e, 0xa6, 0x89, 0xd0, 0xed, 0xf5, 0x77, 0xf7, 0xdb, 0xda, 0x27, 0x94, + 0x27, 0x4d, 0xc3, 0x46, 0xbb, 0xdb, 0x69, 0x6a, 0x8f, 0xf5, 0xdf, 0x82, 0xa2, 0xb4, 0x21, 0x31, + 0x9b, 0x56, 0xa7, 0xd3, 0x34, 0xb4, 0x15, 0x2c, 0xaa, 0xdd, 0x7c, 0xd2, 0xd7, 0x32, 0x74, 0x0e, + 0xd5, 0x7a, 0xba, 0xd7, 0xd7, 0x56, 0xf1, 0xb3, 0x7b, 0x88, 0xbd, 0x96, 0xa5, 0xe6, 0x36, 0xf7, + 0x5b, 0x5a, 0x0e, 0xbf, 0xea, 0x9d, 0x7e, 0x4b, 0xcb, 0xd3, 0xbc, 0x69, 0x75, 0x9e, 0xb6, 0x9b, + 0xda, 0x1a, 0x42, 0xf7, 0xeb, 0xc6, 0x33, 0xad, 0xc0, 0x33, 0xdd, 0x6d, 0x7e, 0xa6, 0x15, 0xd9, + 0x1a, 0xac, 0xb6, 0x1f, 0x6a, 0x25, 0x04, 0xed, 0x36, 0x77, 0x0f, 0x0f, 0x34, 0xd0, 0xef, 0x42, + 0xa1, 0x7e, 0x74, 0xb4, 0x8f, 0x26, 0x3a, 0xb6, 0xee, 0xb0, 0xdd, 0xe6, 0xeb, 0x68, 0xa7, 0xdb, + 0xef, 0x77, 0xf7, 0xb5, 0x0c, 0xce, 0xdc, 0x7e, 0xf7, 0x40, 0x5b, 0xd5, 0x5b, 0x50, 0x94, 0x1b, + 0xa6, 0x72, 0x4f, 0xa7, 0x08, 0xb9, 0x03, 0xa3, 0xf9, 0x9c, 0x1f, 0x9e, 0x76, 0x9a, 0x9f, 0x61, + 0x35, 0xf1, 0x0b, 0x33, 0xca, 0x62, 0x41, 0xfc, 0x42, 0x0d, 0x5d, 0xd4, 0x69, 0xb7, 0x3a, 0xcd, + 0xba, 0xa1, 0xe5, 0xf5, 0x8f, 0x52, 0xe7, 0x52, 0x42, 0xb6, 0x60, 0xf1, 0xf5, 0x96, 0x28, 0xbe, + 0xf5, 0xb4, 0xd3, 0x35, 0x9a, 0xfc, 0xe6, 0x8f, 0xe8, 0xc8, 0x55, 0xfd, 0x1d, 0x28, 0x25, 0x72, + 0x11, 0x27, 0x56, 0xc3, 0xe8, 0xf6, 0x7a, 0xbc, 0xdf, 0x57, 0x30, 0x4d, 0x7d, 0xc3, 0xd3, 0x19, + 0xbd, 0x07, 0x9b, 0x52, 0x24, 0x53, 0x50, 0x34, 0xd9, 0x16, 0x5b, 0x90, 0x6f, 0x3b, 0x2f, 0x1d, + 0x4f, 0x46, 0xf7, 0x52, 0x02, 0xa1, 0xdd, 0xc1, 0x17, 0xad, 0xe4, 0x72, 0x26, 0x25, 0x50, 0x07, + 0xeb, 0x28, 0xf7, 0x43, 0x29, 0xaa, 0xfc, 0xf7, 0x32, 0x50, 0x4c, 0x04, 0xfd, 0x6d, 0x58, 0xed, + 0xf7, 0x84, 0x5f, 0x7b, 0xeb, 0xfe, 0xec, 0x3a, 0x7c, 0x5f, 0x7e, 0x19, 0xab, 0xfd, 0x1e, 0x7b, + 0x17, 0xd6, 0xf8, 0x75, 0x36, 0x71, 0x20, 0xb1, 0x95, 0xde, 0x3c, 0xfa, 0x84, 0x33, 0x04, 0x0d, + 0xfb, 0x08, 0x4a, 0x49, 0x6d, 0x85, 0x7f, 0xe3, 0x72, 0x9a, 0x21, 0x41, 0x1b, 0x33, 0x4a, 0xbd, + 0x0d, 0xd5, 0x74, 0x86, 0xec, 0x3a, 0x00, 0xcf, 0x52, 0xf1, 0xeb, 0x28, 0x10, 0x76, 0x15, 0xe4, + 0x2d, 0xbb, 0x5d, 0xaa, 0xd8, 0x7a, 0x72, 0xeb, 0x6e, 0x57, 0xff, 0xcb, 0x59, 0x80, 0x99, 0xaa, + 0x88, 0x1d, 0x91, 0x78, 0x6d, 0xf2, 0xe2, 0x48, 0xf3, 0x0d, 0x28, 0x79, 0x81, 0x65, 0xab, 0xb7, + 0xe1, 0x8b, 0x08, 0xa0, 0xa1, 0x51, 0x6f, 0x9c, 0x94, 0x78, 0x3c, 0x01, 0xbb, 0x04, 0x6b, 0xa3, + 0x20, 0x1c, 0x5b, 0xb1, 0x08, 0xe5, 0x16, 0x29, 0x94, 0xf8, 0xfc, 0x98, 0x0d, 0x15, 0x66, 0x9f, + 0xa2, 0xb9, 0x71, 0x0c, 0x2a, 0x02, 0xd8, 0x46, 0x18, 0x1a, 0x46, 0x8e, 0x3f, 0xf4, 0x82, 0xc8, + 0xb1, 0xcd, 0x01, 0x0f, 0xd1, 0xa8, 0x18, 0x20, 0x41, 0x3b, 0xe7, 0xbc, 0xb5, 0xe1, 0xd8, 0xf5, + 0xad, 0x58, 0x38, 0x9f, 0xa9, 0xb5, 0x12, 0x82, 0xd5, 0xfd, 0x22, 0x0a, 0x84, 0x13, 0x87, 0x9f, + 0xd8, 0x15, 0x11, 0x40, 0xd5, 0x7d, 0x13, 0xc0, 0x89, 0x86, 0xd6, 0x84, 0x67, 0x5e, 0xa2, 0xcc, + 0x4b, 0x02, 0xb2, 0x73, 0xce, 0xda, 0x50, 0xed, 0x0f, 0x70, 0x87, 0x0a, 0xd0, 0x1e, 0x6f, 0x04, + 0x9e, 0x70, 0xaf, 0xdc, 0x9e, 0xd7, 0xa9, 0xef, 0xa7, 0xc9, 0xf8, 0xd1, 0xe2, 0x1c, 0xef, 0xd5, + 0x3a, 0x5c, 0x58, 0x42, 0xf6, 0x8d, 0x62, 0xac, 0x3c, 0x39, 0x3a, 0xf5, 0x38, 0xa6, 0xdb, 0x13, + 0xc9, 0x66, 0x9c, 0x91, 0x31, 0xe0, 0x7c, 0x1f, 0x7e, 0x83, 0xa2, 0x28, 0x44, 0x78, 0x9e, 0x18, + 0xa4, 0x24, 0xec, 0xee, 0x0e, 0x6c, 0x20, 0x72, 0xe4, 0x3a, 0x9e, 0x2d, 0x48, 0x78, 0xf0, 0xff, + 0xfa, 0x30, 0xf0, 0x9e, 0x20, 0x94, 0xe8, 0xf4, 0xff, 0x3f, 0x0f, 0x30, 0x33, 0xc3, 0x52, 0xa7, + 0x9b, 0x99, 0xf4, 0xe9, 0xe6, 0x43, 0xb8, 0x24, 0xae, 0x86, 0x24, 0x47, 0x84, 0xae, 0x6f, 0x0e, + 0x2c, 0x79, 0x90, 0xcc, 0x04, 0x96, 0x9f, 0x12, 0xb6, 0xfc, 0x1d, 0x0b, 0x95, 0xba, 0x0d, 0x95, + 0x27, 0x3e, 0x9f, 0xa4, 0x0f, 0xc2, 0x55, 0x55, 0x61, 0xc6, 0xde, 0x3f, 0x9f, 0xb0, 0xf7, 0xe1, + 0x62, 0xe8, 0x8c, 0x42, 0x27, 0x3a, 0x36, 0xe3, 0x48, 0x2d, 0x8c, 0xc7, 0x6b, 0x6d, 0x0a, 0x64, + 0x3f, 0x4a, 0xca, 0x7a, 0x1f, 0x2e, 0x0a, 0x03, 0x6d, 0xae, 0x7a, 0xfc, 0x0e, 0xf1, 0x26, 0x47, + 0xaa, 0xb5, 0x7b, 0x13, 0x40, 0xd8, 0xa6, 0xf2, 0x4d, 0x89, 0xa2, 0x51, 0xe2, 0x76, 0xa8, 0xb8, + 0x69, 0x49, 0x06, 0xa6, 0x38, 0xfa, 0xe1, 0x09, 0xa6, 0x43, 0x0e, 0xc5, 0x29, 0x1d, 0x53, 0x54, + 0x1f, 0x56, 0xef, 0xd3, 0x9b, 0x19, 0x74, 0xf3, 0x35, 0xb0, 0x1d, 0x83, 0x70, 0xec, 0x3d, 0xb8, + 0xa0, 0x36, 0x5b, 0x5e, 0xfb, 0x2e, 0x53, 0x45, 0xb4, 0x59, 0x43, 0x0d, 0x7e, 0x01, 0xfc, 0x1d, + 0x60, 0x4a, 0xcd, 0x25, 0x75, 0x85, 0xa8, 0x37, 0x92, 0x6a, 0x0b, 0xe2, 0xb7, 0x81, 0xaa, 0xc8, + 0xbd, 0xc2, 0xeb, 0x8b, 0xd6, 0x18, 0x22, 0xc9, 0x83, 0xfc, 0x3e, 0x5c, 0x9c, 0xb5, 0xce, 0xb4, + 0x62, 0x33, 0x3e, 0x76, 0x4c, 0xc7, 0xb7, 0xe9, 0x3e, 0x4f, 0xd1, 0xd8, 0x4c, 0x1a, 0x5a, 0x8f, + 0xfb, 0xc7, 0x0e, 0xda, 0x53, 0x8a, 0x7f, 0x6c, 0xe3, 0xab, 0xfd, 0x63, 0x1f, 0x43, 0x2d, 0x75, + 0xe4, 0xa9, 0x76, 0x37, 0xbf, 0x0f, 0xb7, 0xa5, 0x1e, 0x74, 0x26, 0x3d, 0x7e, 0x0f, 0x36, 0x8f, + 0xad, 0xc8, 0x4c, 0xf1, 0x92, 0xdb, 0xae, 0x68, 0x6c, 0x1c, 0x5b, 0xd1, 0x81, 0xc2, 0xa3, 0xff, + 0x41, 0x06, 0xaa, 0x69, 0xc3, 0x94, 0xdf, 0x87, 0xf0, 0xa6, 0x63, 0x9f, 0x47, 0x37, 0xe4, 0x0d, + 0x99, 0xc4, 0xb5, 0x30, 0x39, 0x31, 0x79, 0x4a, 0xae, 0x85, 0xc9, 0x49, 0x83, 0xd2, 0xec, 0x3b, + 0x50, 0x98, 0x9c, 0x70, 0xe1, 0xf0, 0xaa, 0xd9, 0xb7, 0x36, 0xe1, 0x61, 0xa5, 0xdf, 0x81, 0xc2, + 0x54, 0x90, 0xe6, 0x5e, 0x45, 0x3a, 0x25, 0x52, 0xfd, 0x9f, 0xac, 0x42, 0x45, 0x75, 0xc9, 0x7c, + 0x9d, 0x93, 0xd0, 0x6f, 0x74, 0x82, 0xbd, 0x4d, 0x51, 0x66, 0x26, 0xc5, 0xb0, 0x62, 0x3f, 0xf1, + 0x63, 0x50, 0x38, 0xb6, 0xa2, 0xfa, 0x34, 0x0e, 0x1a, 0x01, 0x3f, 0x7d, 0x09, 0x3c, 0x19, 0xdb, + 0xca, 0x57, 0x06, 0xca, 0x04, 0x11, 0xd6, 0xfa, 0xbe, 0x08, 0x9d, 0xa7, 0xcb, 0x32, 0x14, 0x3d, + 0x91, 0x5f, 0x98, 0x2f, 0x15, 0x79, 0x57, 0x86, 0x02, 0x23, 0x1e, 0xc2, 0xc6, 0x2c, 0x50, 0x59, + 0x06, 0x5c, 0xcc, 0xb3, 0xac, 0x27, 0x51, 0xca, 0xe2, 0x76, 0xec, 0xba, 0x1b, 0x99, 0x81, 0x67, + 0xcb, 0x1b, 0x11, 0x05, 0xe9, 0xea, 0xee, 0x7a, 0xb6, 0xb8, 0x2f, 0xc5, 0x69, 0x7c, 0xe7, 0x54, + 0xd2, 0x24, 0xee, 0xf0, 0x8e, 0x73, 0x2a, 0x6e, 0x46, 0xfc, 0x51, 0x06, 0x36, 0x17, 0x5c, 0x28, + 0x28, 0x39, 0x67, 0x6f, 0xb5, 0xe0, 0x27, 0xbb, 0x09, 0x95, 0xb1, 0x15, 0x0f, 0x8f, 0xcd, 0x49, + 0xe8, 0x8c, 0xdc, 0x33, 0xf9, 0xe0, 0x0c, 0xc1, 0x0e, 0x08, 0x44, 0x41, 0x28, 0x93, 0x09, 0x39, + 0x8e, 0xc6, 0x6e, 0x2c, 0x04, 0x1f, 0x10, 0xa8, 0x4d, 0xae, 0x65, 0x19, 0xa0, 0x96, 0x7b, 0x45, + 0x80, 0xda, 0x55, 0x28, 0xf9, 0x41, 0x6c, 0x06, 0xbe, 0x39, 0x39, 0x11, 0xd7, 0xa9, 0x0b, 0x7e, + 0x10, 0x77, 0xfd, 0x83, 0x13, 0xfd, 0x1a, 0xac, 0xb5, 0x12, 0x37, 0x4e, 0x12, 0x61, 0x91, 0x15, + 0x2f, 0x2e, 0x04, 0x50, 0x6a, 0xd0, 0xeb, 0x0d, 0xfb, 0xd6, 0x84, 0xdd, 0x83, 0xec, 0xd8, 0x9a, + 0x88, 0x08, 0x8c, 0x5a, 0x72, 0x96, 0xc2, 0xb1, 0xf7, 0xf7, 0xad, 0x09, 0xdf, 0x44, 0x90, 0xe8, + 0xea, 0xc7, 0x50, 0x94, 0x80, 0x6f, 0xb4, 0x5d, 0xfc, 0xf3, 0x55, 0x28, 0xed, 0xaa, 0x6e, 0x5b, + 0xb4, 0x68, 0xe3, 0x70, 0xea, 0xa3, 0xee, 0x25, 0xef, 0xa1, 0x0f, 0x2d, 0xbf, 0x2f, 0x40, 0x72, + 0x9a, 0xae, 0x7e, 0xc5, 0x34, 0xbd, 0x06, 0x10, 0x92, 0x17, 0x83, 0x1c, 0x19, 0xd9, 0x24, 0xda, + 0xaf, 0x65, 0xb7, 0xec, 0xb3, 0xe5, 0xc7, 0xf9, 0xb9, 0xaf, 0x7f, 0x9c, 0x9f, 0x5f, 0x7a, 0x9c, + 0x7f, 0x67, 0xb6, 0x55, 0xe0, 0x74, 0xc5, 0x82, 0x4b, 0x7c, 0xc3, 0x9a, 0x24, 0x41, 0xff, 0x58, + 0xfa, 0x77, 0xa1, 0x2a, 0x5b, 0x27, 0xf2, 0x83, 0xd4, 0x3d, 0x03, 0x81, 0xe3, 0x8e, 0xe0, 0xf5, + 0x58, 0x4d, 0xa6, 0x97, 0x5f, 0xf9, 0x35, 0x71, 0x0e, 0x7f, 0x25, 0x03, 0x4c, 0x58, 0xdd, 0x4f, + 0xa6, 0x9e, 0xd7, 0x77, 0xce, 0x68, 0x95, 0xdf, 0x83, 0x4d, 0xe1, 0xc5, 0x55, 0xa2, 0x84, 0xc4, + 0xc1, 0x1a, 0x47, 0xcc, 0x0e, 0xd6, 0x96, 0x5d, 0x05, 0x5b, 0x5d, 0x7a, 0x15, 0x6c, 0xf9, 0x15, + 0xb3, 0x1b, 0x50, 0x56, 0x2f, 0x52, 0x71, 0xd5, 0x0a, 0xac, 0xd9, 0x1d, 0xaa, 0x7f, 0xb9, 0x0a, + 0x30, 0xf3, 0x0c, 0xfc, 0xaa, 0x63, 0x31, 0x96, 0x0c, 0x49, 0x76, 0xd9, 0x90, 0xdc, 0x05, 0x4d, + 0xa5, 0x53, 0x6e, 0xf4, 0x55, 0x67, 0x84, 0x52, 0x65, 0x71, 0x23, 0xf5, 0xd6, 0x15, 0x05, 0x5e, + 0x89, 0xf3, 0x66, 0x11, 0x95, 0x45, 0xf2, 0x54, 0xec, 0xc6, 0x45, 0x37, 0xe2, 0xf2, 0x95, 0x7d, + 0x0a, 0x57, 0x12, 0x4e, 0xf3, 0xd4, 0x8d, 0x8f, 0x83, 0x69, 0x2c, 0xbc, 0x08, 0x91, 0x90, 0x38, + 0x97, 0x64, 0x4e, 0x2f, 0x38, 0x9a, 0x4b, 0x91, 0x08, 0x95, 0xee, 0xd1, 0xd4, 0xf3, 0xcc, 0xd8, + 0x39, 0x8b, 0xc5, 0xd5, 0xf6, 0x5a, 0xca, 0xa9, 0xa2, 0x0c, 0xaf, 0x51, 0x1c, 0x89, 0x84, 0xfe, + 0x77, 0xb3, 0x90, 0xff, 0xf1, 0xd4, 0x09, 0xcf, 0xd9, 0xc7, 0x50, 0x8a, 0xe2, 0x71, 0xac, 0x1e, + 0x6e, 0x5e, 0xe1, 0x19, 0x10, 0x9e, 0xce, 0x26, 0x9d, 0xb1, 0xe3, 0xc7, 0xdc, 0xdb, 0x88, 0xb4, + 0xb4, 0x99, 0x6c, 0x41, 0x3e, 0x8a, 0x9d, 0x49, 0x24, 0xa2, 0xa8, 0x78, 0x82, 0x6d, 0x43, 0xde, + 0x0f, 0x6c, 0x27, 0x4a, 0xc7, 0x4a, 0x75, 0x50, 0x7b, 0xe0, 0x08, 0xa6, 0xc3, 0x5a, 0x32, 0xe2, + 0x0b, 0x07, 0x8c, 0x1c, 0x43, 0xd1, 0xef, 0x8e, 0x65, 0xbb, 0xfe, 0x91, 0xbc, 0x21, 0x99, 0xa4, + 0x71, 0x9b, 0x24, 0x65, 0xdd, 0x3a, 0x92, 0xd7, 0x95, 0x45, 0x92, 0x6d, 0x43, 0x19, 0x3f, 0x5f, + 0x84, 0x6e, 0xec, 0xf4, 0x1e, 0x49, 0x49, 0xad, 0x80, 0x50, 0xd5, 0xb6, 0x9d, 0xd8, 0x19, 0xc6, + 0xbd, 0x2f, 0x45, 0x00, 0x14, 0xc5, 0xc9, 0x48, 0x08, 0xfb, 0x2e, 0xb0, 0x81, 0x35, 0x3c, 0x39, + 0x0a, 0x83, 0xa9, 0x6f, 0x9b, 0x5f, 0x4e, 0x9d, 0xd0, 0x75, 0x64, 0xc0, 0x53, 0x59, 0xe9, 0x14, + 0x63, 0x73, 0x46, 0xf6, 0x63, 0x4e, 0xa5, 0xdb, 0xb0, 0x9e, 0xea, 0xaa, 0x05, 0xef, 0x4e, 0xaf, + 0xd9, 0x6e, 0x36, 0xfa, 0xdc, 0x2c, 0x14, 0x2e, 0x85, 0x55, 0xd5, 0x25, 0x91, 0x55, 0x7c, 0x15, + 0x39, 0xc5, 0x76, 0xcc, 0x93, 0xa7, 0xa3, 0x69, 0x3c, 0x6d, 0x6a, 0x6b, 0xfa, 0x1f, 0xae, 0xc2, + 0x66, 0x3f, 0xb4, 0xfc, 0xc8, 0xe2, 0x9a, 0x84, 0x1f, 0x87, 0x81, 0xc7, 0xbe, 0x0b, 0xc5, 0x78, + 0xe8, 0xa9, 0x43, 0x78, 0x43, 0x0a, 0x8c, 0x39, 0xd2, 0xfb, 0xfd, 0x21, 0x77, 0x1b, 0x17, 0x62, + 0xfe, 0xc1, 0xde, 0x83, 0xfc, 0xc0, 0x39, 0x72, 0x7d, 0x21, 0x33, 0x2f, 0xce, 0x33, 0xee, 0x20, + 0x72, 0x6f, 0xc5, 0xe0, 0x54, 0xec, 0x7d, 0x58, 0x1b, 0x06, 0x63, 0xb9, 0xf1, 0xcc, 0x6e, 0xda, + 0x28, 0x05, 0x21, 0x76, 0x6f, 0xc5, 0x10, 0x74, 0xec, 0x63, 0x28, 0x86, 0x81, 0xe7, 0x61, 0x8f, + 0x89, 0x2d, 0xa9, 0x36, 0xcf, 0x63, 0x08, 0xfc, 0xde, 0x8a, 0x91, 0xd0, 0xea, 0xf7, 0xa1, 0x20, + 0x2a, 0x8b, 0x1d, 0xb0, 0xd3, 0x7c, 0xda, 0x12, 0x1d, 0xd9, 0xe8, 0xee, 0xef, 0xb7, 0xfa, 0xfc, + 0x1a, 0xa1, 0xd1, 0x6d, 0xb7, 0x77, 0xea, 0x8d, 0x67, 0xda, 0xea, 0x4e, 0x11, 0xd6, 0xb8, 0x83, + 0x50, 0xff, 0xed, 0x0c, 0x6c, 0xcc, 0x35, 0x80, 0x3d, 0x86, 0xdc, 0x18, 0x35, 0x5b, 0xde, 0x3d, + 0xb7, 0x97, 0xb6, 0x52, 0x49, 0x73, 0x7d, 0x17, 0x39, 0xf4, 0x4f, 0xa1, 0x9a, 0x86, 0x2b, 0x1e, + 0x84, 0x75, 0x28, 0x19, 0xcd, 0xfa, 0xae, 0xd9, 0xed, 0xa0, 0xdd, 0x8e, 0x76, 0x3c, 0x25, 0x5f, + 0x18, 0x2d, 0x32, 0xfa, 0x7f, 0x03, 0xb4, 0xf9, 0x8e, 0x61, 0x4f, 0xd1, 0x76, 0x19, 0x4f, 0x3c, + 0x87, 0x54, 0x44, 0x65, 0xc8, 0xae, 0x2f, 0xe9, 0x49, 0x41, 0x46, 0x23, 0x56, 0x1d, 0xa6, 0xd2, + 0xfa, 0x6f, 0x02, 0x5b, 0xec, 0xc1, 0x5f, 0x5d, 0xf6, 0xff, 0x2d, 0x03, 0xb9, 0x03, 0xcf, 0xf2, + 0xd9, 0x2d, 0xc8, 0xd3, 0xf3, 0x17, 0x42, 0xf2, 0xaa, 0xeb, 0x00, 0xa7, 0x05, 0xe1, 0xd8, 0x3b, + 0x90, 0x8d, 0x87, 0xf2, 0xf6, 0xe2, 0xe5, 0x57, 0x4c, 0xbe, 0xbd, 0x15, 0x03, 0xa9, 0xd8, 0x5d, + 0xc8, 0xda, 0xb6, 0x8c, 0x1a, 0x16, 0x3e, 0x05, 0xb4, 0x28, 0x77, 0x9d, 0x91, 0xeb, 0xbb, 0xe2, + 0xb9, 0x0e, 0x24, 0x61, 0x6f, 0x41, 0xd6, 0x1e, 0x7a, 0xe9, 0x10, 0x70, 0x6e, 0x7b, 0x26, 0x19, + 0xda, 0x43, 0x0f, 0x35, 0xb0, 0x38, 0x3c, 0x37, 0xc3, 0xa9, 0x4f, 0x61, 0x50, 0x91, 0xb0, 0x8a, + 0xca, 0xa8, 0x7f, 0x4c, 0x29, 0x96, 0x2a, 0x12, 0xd7, 0xa0, 0x26, 0xa1, 0x33, 0xb1, 0xc2, 0xc4, + 0x1e, 0x72, 0xa3, 0x03, 0x0e, 0xd8, 0x59, 0x03, 0x7a, 0x55, 0x50, 0x7f, 0x97, 0xde, 0x66, 0x40, + 0xc5, 0x5a, 0x97, 0x5f, 0x4b, 0x1e, 0xa0, 0x12, 0x18, 0xfd, 0xcf, 0xb3, 0x50, 0x56, 0xea, 0xc3, + 0x3e, 0x84, 0xa2, 0x9d, 0x5e, 0x88, 0x57, 0x16, 0x2a, 0x7d, 0x7f, 0x57, 0x2e, 0x41, 0x5b, 0x4c, + 0x6f, 0x3a, 0x93, 0x88, 0xcd, 0x97, 0x56, 0xe8, 0xf2, 0x17, 0x79, 0x56, 0xd5, 0xc3, 0x81, 0x9e, + 0x13, 0x3f, 0x97, 0x98, 0xbd, 0x15, 0xa3, 0x12, 0x29, 0x69, 0xd2, 0xfe, 0x45, 0x93, 0xb2, 0xa9, + 0x97, 0x8d, 0x38, 0x70, 0x6f, 0xc5, 0x90, 0x78, 0x24, 0x75, 0xce, 0x9c, 0xe1, 0x34, 0x96, 0xda, + 0xff, 0xba, 0x6c, 0x10, 0x01, 0xe9, 0x79, 0x35, 0xfe, 0xc9, 0x1e, 0xa2, 0x9c, 0xb4, 0x3c, 0x2f, + 0x20, 0x35, 0x2b, 0xaf, 0xba, 0xea, 0x77, 0x13, 0x38, 0x7f, 0xce, 0x4d, 0xa6, 0xd8, 0x1d, 0xc8, + 0x07, 0xf1, 0xb1, 0x23, 0x75, 0x6a, 0xf9, 0xca, 0x03, 0x82, 0x76, 0x1b, 0x6d, 0x9c, 0x29, 0x84, + 0xd6, 0x7f, 0x9e, 0x81, 0x82, 0xe8, 0x01, 0xb6, 0x09, 0xeb, 0xbd, 0x66, 0xdf, 0x7c, 0x5e, 0x37, + 0x5a, 0xf5, 0x9d, 0x76, 0x53, 0x44, 0xae, 0x3f, 0x35, 0xea, 0x1d, 0x21, 0x27, 0x8d, 0xe6, 0xf3, + 0xee, 0xb3, 0x26, 0x77, 0xcb, 0xed, 0x36, 0x3b, 0x9f, 0x6b, 0x59, 0xee, 0x9a, 0x6e, 0x1e, 0xd4, + 0x0d, 0x94, 0x92, 0x65, 0x28, 0x34, 0x3f, 0x6b, 0x36, 0x0e, 0x49, 0x4c, 0x56, 0x01, 0x76, 0x9b, + 0xf5, 0x76, 0xbb, 0xdb, 0x40, 0xb1, 0xb9, 0xc6, 0x18, 0x54, 0x1b, 0x46, 0xb3, 0xde, 0x6f, 0x9a, + 0xf5, 0x46, 0xa3, 0x7b, 0xd8, 0xe9, 0x6b, 0x05, 0x2c, 0xb1, 0xde, 0xee, 0x37, 0x8d, 0x04, 0x44, + 0x2f, 0xef, 0xec, 0x1a, 0xdd, 0x83, 0x04, 0x52, 0xda, 0x29, 0xa1, 0x25, 0x46, 0x63, 0xa5, 0xff, + 0xfd, 0x4d, 0xa8, 0xa6, 0xa7, 0x26, 0xfb, 0x04, 0x8a, 0xb6, 0x9d, 0x1a, 0xe3, 0x6b, 0xcb, 0xa6, + 0xf0, 0xfd, 0x5d, 0x5b, 0x0e, 0x33, 0xff, 0x60, 0x37, 0xe5, 0x42, 0x5a, 0x5d, 0x58, 0x48, 0x72, + 0x19, 0xfd, 0x00, 0x36, 0xc4, 0x2b, 0x09, 0xb6, 0x15, 0x5b, 0x03, 0x2b, 0x72, 0xd2, 0xab, 0xa4, + 0x41, 0xc8, 0x5d, 0x81, 0xdb, 0x5b, 0x31, 0xaa, 0xc3, 0x14, 0x84, 0x7d, 0x0f, 0xaa, 0x16, 0x19, + 0xdb, 0x09, 0x7f, 0x4e, 0x55, 0x20, 0xeb, 0x88, 0x53, 0xd8, 0xd7, 0x2d, 0x15, 0x80, 0x13, 0xd1, + 0x0e, 0x83, 0xc9, 0x8c, 0x39, 0x9f, 0x3a, 0xa5, 0x0a, 0x83, 0x89, 0xc2, 0x5b, 0xb1, 0x95, 0x34, + 0xfb, 0x18, 0x2a, 0xa2, 0xe6, 0x33, 0x87, 0x43, 0xb2, 0x64, 0x79, 0xb5, 0x49, 0x21, 0xdc, 0x5b, + 0x31, 0xca, 0xc3, 0x59, 0x92, 0x3d, 0x42, 0x2d, 0x70, 0xa6, 0x3e, 0x17, 0xd4, 0xb9, 0x46, 0xb5, + 0x95, 0x5c, 0x60, 0x25, 0x29, 0xf6, 0x3e, 0x00, 0xd5, 0x93, 0xf3, 0x14, 0x53, 0xf1, 0x1e, 0x61, + 0x30, 0x91, 0x2c, 0x25, 0x5b, 0x26, 0x94, 0xea, 0x71, 0x77, 0x51, 0x69, 0xb1, 0x7a, 0xe4, 0x32, + 0x9a, 0x55, 0x8f, 0x7b, 0x9a, 0x92, 0xea, 0x71, 0x36, 0x58, 0xa8, 0x9e, 0xe4, 0xe2, 0xd5, 0xe3, + 0x4c, 0xb2, 0x7a, 0x9c, 0xa7, 0x3c, 0x5f, 0x3d, 0xc9, 0x42, 0xd5, 0xe3, 0x1c, 0xdf, 0x5b, 0xd0, + 0xfb, 0x2b, 0xaf, 0xd4, 0xfb, 0x71, 0xd8, 0xd2, 0x9a, 0xff, 0xf7, 0xa0, 0x1a, 0x1d, 0x07, 0xa7, + 0x8a, 0x00, 0x59, 0x57, 0xb9, 0x7b, 0xc7, 0xc1, 0xa9, 0x2a, 0x41, 0xd6, 0x23, 0x15, 0x80, 0xb5, + 0xe5, 0x4d, 0xa4, 0xeb, 0xd9, 0x55, 0xb5, 0xb6, 0xd4, 0xc2, 0xe7, 0xae, 0x73, 0x8a, 0xb5, 0xb5, + 0x64, 0x02, 0x3b, 0x65, 0xe6, 0x7c, 0x89, 0x84, 0x3b, 0x25, 0x15, 0xf5, 0x20, 0x4a, 0x82, 0xc4, + 0x0d, 0x13, 0xe1, 0xdc, 0x9a, 0xfa, 0x2a, 0x9b, 0xa6, 0xce, 0xad, 0x43, 0x3f, 0xc5, 0x58, 0xe1, + 0xa4, 0x82, 0x75, 0xb6, 0x2a, 0x22, 0xe7, 0xcb, 0xa9, 0xe3, 0x0f, 0x1d, 0x11, 0x0d, 0x95, 0x5a, + 0x15, 0x3d, 0x81, 0x9b, 0xad, 0x0a, 0x09, 0x49, 0xe6, 0x75, 0xc2, 0xce, 0xe6, 0xe7, 0xb5, 0xc2, + 0x4c, 0xf3, 0x3a, 0x61, 0x4d, 0x16, 0x54, 0xc2, 0x7b, 0x61, 0x61, 0x41, 0x29, 0xcc, 0x7c, 0x41, + 0x25, 0xdc, 0x8f, 0x40, 0xcc, 0x26, 0xde, 0xb9, 0xa9, 0x98, 0x29, 0x5e, 0x6b, 0xd1, 0xbb, 0x30, + 0x4c, 0x52, 0x38, 0x57, 0x43, 0x07, 0xed, 0x0c, 0x31, 0x15, 0x2e, 0xaa, 0x73, 0xd5, 0x20, 0x4c, + 0xb2, 0x94, 0xc2, 0x59, 0x52, 0x29, 0x6c, 0xe2, 0xc6, 0x61, 0xcd, 0x5e, 0x2c, 0xec, 0xc0, 0x8d, + 0xc3, 0x59, 0x61, 0x98, 0x62, 0xef, 0x01, 0x4d, 0x43, 0xce, 0xe2, 0xa8, 0xa2, 0x1b, 0xbb, 0x45, + 0x30, 0x14, 0x6d, 0xf1, 0x8d, 0x93, 0x45, 0x94, 0x31, 0xb4, 0x87, 0xb5, 0x91, 0x3a, 0x59, 0x78, + 0x11, 0x8d, 0xdd, 0x06, 0x4e, 0x16, 0x4e, 0xd4, 0xb0, 0x87, 0xec, 0x1e, 0x10, 0x37, 0xd1, 0x1f, + 0xa5, 0x5e, 0x11, 0x0a, 0x83, 0x09, 0xa7, 0x2e, 0x20, 0x01, 0xd2, 0x62, 0x0b, 0xbc, 0xc0, 0x97, + 0x0d, 0x3f, 0x4e, 0xb5, 0x00, 0x11, 0x89, 0x30, 0x18, 0x26, 0x29, 0xfd, 0xff, 0x5e, 0x83, 0x82, + 0x90, 0xb5, 0xec, 0x02, 0x6c, 0x08, 0x91, 0xbf, 0x5b, 0xef, 0xd7, 0x77, 0xea, 0x3d, 0x54, 0xd2, + 0x18, 0x54, 0xb9, 0xcc, 0x4f, 0x60, 0x19, 0xdc, 0x07, 0x48, 0xe8, 0x27, 0xa0, 0x55, 0xdc, 0x07, + 0x04, 0x2f, 0x7f, 0xad, 0x2d, 0xcb, 0x36, 0xa0, 0xcc, 0x19, 0x39, 0x80, 0x2e, 0xd2, 0x11, 0x17, + 0x4f, 0xe7, 0x15, 0x16, 0x7e, 0x4a, 0xb5, 0x36, 0x63, 0xe1, 0x80, 0x42, 0xc2, 0x22, 0x8f, 0xb1, + 0x18, 0x54, 0xfb, 0xc6, 0x61, 0xa7, 0x31, 0x2b, 0xa7, 0x44, 0x97, 0x9f, 0x78, 0x36, 0xcf, 0x5b, + 0xcd, 0x17, 0x1a, 0x20, 0x13, 0xcf, 0x85, 0xd2, 0x65, 0x54, 0x33, 0x29, 0x13, 0x4a, 0x56, 0xd8, + 0x65, 0xb8, 0xd0, 0xdb, 0xeb, 0xbe, 0x30, 0x39, 0x53, 0xd2, 0x84, 0x75, 0xb6, 0x05, 0x9a, 0x82, + 0xe0, 0xd9, 0x57, 0xb1, 0x48, 0x82, 0x4a, 0xc2, 0x9e, 0xb6, 0x41, 0x07, 0xc1, 0x08, 0xeb, 0xf3, + 0x7d, 0x57, 0xc3, 0xa6, 0x70, 0xd6, 0x6e, 0xfb, 0x70, 0xbf, 0xd3, 0xd3, 0x36, 0xb1, 0x12, 0x04, + 0xe1, 0x35, 0x67, 0x49, 0x36, 0xb3, 0xdd, 0xfa, 0x02, 0x6d, 0xe0, 0x08, 0x7b, 0x51, 0x37, 0x3a, + 0xad, 0xce, 0xd3, 0x9e, 0xb6, 0x95, 0xe4, 0xdc, 0x34, 0x8c, 0xae, 0xd1, 0xd3, 0x2e, 0x26, 0x80, + 0x5e, 0xbf, 0xde, 0x3f, 0xec, 0x69, 0x97, 0x92, 0x5a, 0x1e, 0x18, 0xdd, 0x46, 0xb3, 0xd7, 0x6b, + 0xb7, 0x7a, 0x7d, 0xed, 0x32, 0xbb, 0x08, 0x9b, 0xb3, 0x1a, 0x49, 0xe2, 0x9a, 0x52, 0x51, 0xe3, + 0x69, 0xb3, 0xaf, 0x5d, 0x49, 0xaa, 0xd1, 0xe8, 0xb6, 0xdb, 0x75, 0x3a, 0xc2, 0xbc, 0x8a, 0x44, + 0x74, 0x96, 0x2b, 0x5a, 0xf3, 0x06, 0xd6, 0xeb, 0xb0, 0xa3, 0x82, 0xae, 0x29, 0x53, 0xa3, 0xd7, + 0xfc, 0xf1, 0x61, 0xb3, 0xd3, 0x68, 0x6a, 0x6f, 0xce, 0xa6, 0x46, 0x02, 0xbb, 0x9e, 0x4c, 0x8d, + 0x04, 0x74, 0x23, 0x29, 0x53, 0x82, 0x7a, 0xda, 0x36, 0xe6, 0x27, 0xea, 0xd1, 0xe9, 0x34, 0x1b, + 0x7d, 0x6c, 0xeb, 0xcd, 0xa4, 0x17, 0x0f, 0x0f, 0x9e, 0x1a, 0xf5, 0xdd, 0xa6, 0xa6, 0x23, 0xc4, + 0x68, 0x76, 0xea, 0xfb, 0x72, 0xb4, 0x6f, 0x29, 0xa3, 0x7d, 0xd0, 0xea, 0x1b, 0xda, 0xed, 0x64, + 0x74, 0x29, 0xf9, 0x16, 0x7b, 0x03, 0x2e, 0xab, 0xf3, 0xd0, 0x7c, 0xd1, 0xea, 0xef, 0x89, 0x13, + 0xd7, 0x3b, 0xfc, 0xe4, 0x90, 0x90, 0x8d, 0xdd, 0x06, 0x3f, 0x5a, 0x26, 0x5e, 0x4c, 0xdd, 0xdd, + 0xa9, 0xd0, 0xa3, 0xbb, 0x42, 0x01, 0xd1, 0x7f, 0x04, 0x4c, 0x7d, 0x7f, 0x52, 0x3c, 0xc4, 0xc4, + 0x20, 0x37, 0x0a, 0x83, 0xb1, 0xbc, 0xd4, 0x8e, 0xdf, 0x68, 0x39, 0x4f, 0xa6, 0x03, 0x3a, 0xd1, + 0x9c, 0x5d, 0x5a, 0x55, 0x41, 0xfa, 0xdf, 0xcc, 0x40, 0x35, 0xad, 0x7c, 0x90, 0xdb, 0x73, 0x64, + 0xfa, 0x41, 0xcc, 0x5f, 0xf8, 0x89, 0x92, 0x67, 0x21, 0x47, 0x9d, 0x20, 0xa6, 0x27, 0x7e, 0xc8, + 0x90, 0x4f, 0x74, 0x09, 0x9e, 0x6b, 0x92, 0x66, 0x2d, 0xb8, 0x90, 0x7a, 0xc2, 0x33, 0xf5, 0xbe, + 0x52, 0x2d, 0x79, 0x7a, 0x6f, 0xae, 0xfe, 0x06, 0x8b, 0x16, 0xdb, 0x24, 0xae, 0x1e, 0xe7, 0x66, + 0x57, 0x8f, 0xf7, 0x60, 0x3d, 0xa5, 0xeb, 0x90, 0xff, 0x65, 0x94, 0xae, 0x69, 0xd1, 0x1d, 0xbd, + 0xbe, 0x9a, 0xfa, 0x5f, 0xcf, 0x40, 0x45, 0xd5, 0x7c, 0xbe, 0x75, 0x4e, 0x74, 0x4d, 0x47, 0x7c, + 0x9b, 0xae, 0x2d, 0x5f, 0xf6, 0x91, 0xa0, 0x16, 0x3d, 0x36, 0xce, 0x3d, 0xc8, 0x4f, 0x4e, 0x7a, + 0x49, 0x73, 0x54, 0x10, 0xbb, 0x0e, 0x40, 0x97, 0x16, 0x9f, 0x3c, 0x43, 0x02, 0x71, 0xd1, 0x67, + 0x06, 0xd1, 0x6f, 0x40, 0xe9, 0xc9, 0x89, 0x0c, 0x65, 0x51, 0xdf, 0xb9, 0x2a, 0xf1, 0x9b, 0xce, + 0xfa, 0x9f, 0x64, 0xa0, 0x3a, 0x7b, 0x13, 0x84, 0xce, 0x95, 0xf9, 0xd3, 0xaf, 0x7c, 0x3a, 0xac, + 0xda, 0x83, 0xd9, 0x3b, 0xe4, 0xab, 0xea, 0x3b, 0xe4, 0xb7, 0x44, 0x66, 0x59, 0x55, 0xe4, 0x27, + 0x65, 0x89, 0x7b, 0xd4, 0x8f, 0xa0, 0x82, 0xff, 0x0d, 0x67, 0xe4, 0x84, 0xa1, 0x23, 0x5f, 0xc1, + 0x5d, 0x20, 0x4e, 0x11, 0x91, 0x8d, 0xe7, 0x8c, 0x84, 0xaa, 0xb9, 0xf4, 0xd9, 0x12, 0x7a, 0x4e, + 0xe7, 0x3f, 0x67, 0xa1, 0xac, 0xe8, 0x91, 0x5f, 0x6b, 0xfa, 0x5d, 0x83, 0xd2, 0xec, 0x11, 0x0d, + 0x71, 0x79, 0x35, 0x01, 0xa4, 0xc6, 0x2a, 0x3b, 0x37, 0x56, 0x35, 0x28, 0x88, 0xe0, 0x56, 0xe1, + 0xfc, 0x95, 0xc9, 0xb4, 0x9b, 0x35, 0xff, 0x9a, 0x53, 0x8e, 0x0f, 0xa0, 0xa2, 0xf8, 0x48, 0x23, + 0x71, 0xc1, 0x73, 0x9e, 0xbe, 0x3c, 0xf3, 0x97, 0x46, 0xec, 0x22, 0xac, 0x8d, 0x4e, 0x4c, 0x7b, + 0xc0, 0x6f, 0xf5, 0x95, 0x8c, 0xfc, 0xe8, 0x64, 0x77, 0x40, 0x67, 0x40, 0xa3, 0x44, 0x75, 0xe2, + 0x9e, 0xab, 0xe2, 0x48, 0x2a, 0x48, 0x77, 0xa1, 0x30, 0x3a, 0x51, 0x6f, 0xe7, 0x2d, 0x74, 0xf9, + 0xda, 0xe8, 0x84, 0xae, 0xf3, 0x3d, 0x80, 0x2d, 0xb1, 0x7f, 0x5b, 0x91, 0xc9, 0xdf, 0x09, 0xa0, + 0xc7, 0x55, 0xf8, 0xab, 0x57, 0x9b, 0x1c, 0x57, 0x8f, 0x7a, 0x84, 0xc1, 0x19, 0xa7, 0x43, 0x45, + 0x99, 0x80, 0xfc, 0x15, 0x9a, 0x92, 0x91, 0x82, 0xb1, 0xc7, 0x50, 0x19, 0x9d, 0xf0, 0x01, 0xed, + 0x07, 0xfb, 0x8e, 0x08, 0xa5, 0xdf, 0x9a, 0x1f, 0x4a, 0x3a, 0xf8, 0x4f, 0x51, 0xb2, 0x4b, 0xb0, + 0x66, 0x58, 0xa7, 0xbd, 0x1f, 0xb7, 0x49, 0x89, 0x2c, 0x19, 0x22, 0xf5, 0xa3, 0x5c, 0xb1, 0xaa, + 0x6d, 0xe8, 0xff, 0x20, 0x03, 0xd5, 0x99, 0x0d, 0x80, 0x8b, 0x90, 0xdd, 0x53, 0xdf, 0x6c, 0xae, + 0xcd, 0x9b, 0x09, 0x48, 0x72, 0xbf, 0x7f, 0x3e, 0xe1, 0x2f, 0x1b, 0x2e, 0x7b, 0x39, 0x68, 0x99, + 0xd3, 0x3a, 0xbb, 0xf4, 0xb5, 0xd8, 0xa7, 0x90, 0xed, 0x9f, 0x4f, 0xb8, 0xbf, 0x09, 0xb7, 0x44, + 0x6e, 0x9b, 0xf2, 0xcd, 0x90, 0x22, 0x4b, 0x9e, 0x35, 0x3f, 0xe7, 0x97, 0xed, 0x0f, 0x8c, 0xd6, + 0x7e, 0xdd, 0xf8, 0x9c, 0x82, 0x86, 0x48, 0x69, 0x78, 0xd2, 0x35, 0x9a, 0xad, 0xa7, 0x1d, 0x02, + 0xe4, 0xc8, 0x1b, 0x35, 0xab, 0x62, 0xdd, 0xb6, 0x9f, 0x9c, 0xa8, 0x8f, 0xae, 0x64, 0x52, 0x8f, + 0xae, 0xa4, 0xef, 0xe7, 0xae, 0xce, 0xdf, 0xcf, 0x65, 0xc9, 0x2a, 0x4c, 0x96, 0x34, 0x7b, 0x1b, + 0x72, 0xa3, 0x13, 0xe7, 0x3c, 0x6d, 0xe8, 0xa5, 0x17, 0x10, 0x11, 0xe8, 0xbf, 0xc8, 0x00, 0x4b, + 0x55, 0x84, 0xdb, 0x1e, 0xdf, 0xb6, 0x2e, 0x9f, 0x40, 0x4d, 0x3c, 0x03, 0xc8, 0xa9, 0x14, 0x2f, + 0xb9, 0xe8, 0xd2, 0x8b, 0xc1, 0x2c, 0x8a, 0x73, 0xf6, 0xb8, 0x11, 0x7b, 0x00, 0xfc, 0x1d, 0x36, + 0x8a, 0x13, 0xc9, 0xbd, 0xc2, 0x4e, 0x34, 0x66, 0x34, 0xb3, 0x87, 0xd7, 0xd4, 0x07, 0xe5, 0xb8, + 0x83, 0x7d, 0x63, 0x36, 0x6a, 0xb4, 0xe6, 0xf5, 0xdf, 0xcf, 0xc0, 0x85, 0xf4, 0x84, 0xf8, 0xe5, + 0x5a, 0x99, 0x7e, 0x3d, 0x2f, 0x3b, 0xff, 0x7a, 0xde, 0xb2, 0xf9, 0x94, 0x5b, 0x3a, 0x9f, 0x7e, + 0x27, 0x03, 0x5b, 0x4a, 0xef, 0xcf, 0xac, 0xc5, 0xbf, 0xa0, 0x9a, 0x29, 0x8f, 0xe8, 0xe5, 0x52, + 0x8f, 0xe8, 0xe9, 0x7f, 0x98, 0x81, 0x4b, 0x73, 0x35, 0x31, 0x9c, 0xbf, 0xd0, 0xba, 0xa4, 0x1f, + 0xdb, 0x23, 0x27, 0x3f, 0x0f, 0x65, 0xe5, 0x17, 0x3b, 0x59, 0xfa, 0xf5, 0x3c, 0xba, 0x46, 0xfd, + 0x21, 0x6c, 0xce, 0xea, 0xd8, 0x10, 0x0f, 0x03, 0xde, 0x80, 0xb2, 0xef, 0x9c, 0x9a, 0xf2, 0xd9, + 0x40, 0x11, 0x19, 0xe4, 0x3b, 0xa7, 0x82, 0x40, 0x7f, 0xa2, 0x0a, 0x8c, 0xe4, 0x0d, 0x71, 0xcf, + 0x4e, 0x85, 0x98, 0x04, 0x9e, 0x2d, 0x51, 0x98, 0x9b, 0xd2, 0xa2, 0x82, 0xef, 0x9c, 0xd2, 0x60, + 0x9d, 0x8a, 0x7c, 0xea, 0xb6, 0x2d, 0x8e, 0xd9, 0x97, 0xbd, 0x05, 0x74, 0x05, 0x8a, 0x93, 0x30, + 0xd5, 0x25, 0x85, 0x49, 0xc8, 0x8b, 0xbd, 0x2d, 0xe2, 0x8e, 0x5e, 0x75, 0x24, 0xcf, 0x23, 0x91, + 0xc4, 0x6f, 0x0c, 0xe4, 0x66, 0xbf, 0x31, 0xf0, 0x91, 0x90, 0x15, 0x64, 0x1c, 0xf1, 0x92, 0x35, + 0xc8, 0xba, 0xf6, 0x19, 0x15, 0xbc, 0x6e, 0xe0, 0x27, 0xa9, 0x3b, 0xce, 0x97, 0x22, 0xf4, 0x09, + 0x3f, 0xf5, 0x1d, 0x28, 0x1b, 0x29, 0x4b, 0xb0, 0xa2, 0x38, 0x55, 0xa2, 0xf4, 0x73, 0x29, 0xb3, + 0x0e, 0x32, 0xca, 0x33, 0x9f, 0x4a, 0xa4, 0x47, 0x42, 0x3a, 0x3c, 0xb7, 0xc2, 0xe1, 0xb1, 0x15, + 0xb6, 0x1d, 0xff, 0x28, 0x3e, 0xc6, 0x2e, 0xe7, 0xbe, 0x4e, 0xb5, 0x0b, 0x81, 0x83, 0xe4, 0xd0, + 0x63, 0x2f, 0x7a, 0x44, 0x2e, 0x5f, 0x2f, 0xf7, 0x9d, 0x53, 0xc1, 0xff, 0x26, 0x00, 0xf6, 0xbf, + 0x40, 0xf3, 0x13, 0xb6, 0x52, 0xe0, 0xd9, 0x1c, 0xad, 0x6f, 0x8a, 0xf6, 0x8a, 0x5b, 0x25, 0xbb, + 0xce, 0x48, 0xf7, 0xc4, 0xc8, 0xf3, 0x06, 0x89, 0x4e, 0xf8, 0x56, 0xc3, 0xc8, 0x6e, 0x42, 0x45, + 0x9a, 0xed, 0xf4, 0x42, 0x0f, 0x2f, 0xbe, 0x2c, 0x61, 0x9d, 0xe9, 0x58, 0xff, 0x83, 0x2c, 0x54, + 0xea, 0x3c, 0x08, 0x65, 0x72, 0xde, 0x9d, 0xc4, 0xec, 0x37, 0xe1, 0x62, 0x74, 0xe2, 0x4e, 0xc4, + 0x73, 0xe3, 0x14, 0xfb, 0x41, 0x71, 0xbe, 0xa2, 0x13, 0xef, 0x29, 0x9d, 0x28, 0x58, 0xee, 0xf7, + 0x4e, 0xdc, 0x09, 0x0f, 0x2f, 0x6f, 0xd9, 0x67, 0x14, 0xcb, 0xcd, 0x8f, 0xbe, 0x59, 0xb4, 0x80, + 0xa0, 0x7b, 0xa7, 0x98, 0xfd, 0xe4, 0x44, 0x64, 0x2b, 0x4e, 0xf8, 0x11, 0x78, 0x70, 0xc2, 0x69, + 0xee, 0xc1, 0x26, 0xbf, 0x51, 0xb2, 0xb8, 0x4b, 0x6d, 0x70, 0xc4, 0x6c, 0x7e, 0xf7, 0x60, 0x93, + 0xf2, 0x13, 0xcf, 0xc0, 0x99, 0xc3, 0x60, 0x72, 0x2e, 0x4e, 0xd6, 0xde, 0x7e, 0x45, 0x55, 0x5b, + 0x9c, 0x14, 0x41, 0xe2, 0x09, 0x89, 0x28, 0x0d, 0xbd, 0xda, 0x84, 0xcb, 0xaf, 0x68, 0xd3, 0xeb, + 0x4e, 0xef, 0x8b, 0xca, 0xe9, 0xfd, 0xd5, 0x1d, 0xd8, 0x5a, 0x56, 0xde, 0x37, 0xc9, 0x43, 0xff, + 0xf7, 0x15, 0x80, 0xd9, 0x8c, 0x4d, 0xe9, 0x6c, 0x99, 0x39, 0x9d, 0xed, 0x1b, 0xc5, 0x9f, 0x7c, + 0x08, 0x55, 0xec, 0x2a, 0x73, 0xc6, 0x91, 0x5d, 0xca, 0x51, 0x41, 0xaa, 0xfe, 0xec, 0x92, 0xdd, + 0xe2, 0x89, 0x7f, 0x6e, 0xe9, 0x89, 0xff, 0x07, 0x50, 0xe0, 0xa7, 0x51, 0x91, 0xb8, 0xa4, 0x79, + 0x79, 0x7e, 0xf5, 0xdd, 0x17, 0xa1, 0xea, 0x92, 0x8e, 0x35, 0xa1, 0x8a, 0xf2, 0x31, 0x74, 0xe3, + 0xe3, 0xb1, 0x7a, 0x65, 0xf3, 0xfa, 0x22, 0xa7, 0x24, 0xe3, 0x8f, 0xc7, 0x59, 0x6a, 0x52, 0x51, + 0xf1, 0xe2, 0xb1, 0x70, 0x91, 0x92, 0x8a, 0x57, 0x50, 0x55, 0xbc, 0xfe, 0x98, 0x3b, 0x46, 0x51, + 0xc5, 0x7b, 0x0f, 0x2e, 0x88, 0xeb, 0x33, 0xc8, 0x80, 0xdd, 0x49, 0xf4, 0x3c, 0xd4, 0x50, 0xbc, + 0xac, 0xd2, 0x1f, 0x93, 0x01, 0x84, 0xe4, 0x9f, 0xc1, 0xd6, 0xf0, 0xd8, 0xf2, 0x8f, 0x1c, 0x33, + 0x1e, 0x78, 0x26, 0x3d, 0x5d, 0x6d, 0x8e, 0xad, 0x89, 0xd0, 0x3c, 0xdf, 0x5e, 0xa8, 0x6c, 0x83, + 0x88, 0xfb, 0x03, 0x8f, 0x62, 0xa5, 0x92, 0xb8, 0x90, 0xcd, 0xe1, 0x3c, 0x7c, 0xee, 0x78, 0x16, + 0x16, 0x8e, 0x67, 0xe7, 0x75, 0xd1, 0xf2, 0x12, 0x5d, 0x74, 0xa6, 0x51, 0x56, 0x54, 0x8d, 0x92, + 0xbd, 0x0b, 0x05, 0x71, 0x5b, 0x50, 0x38, 0x47, 0xd9, 0xe2, 0xea, 0x30, 0x24, 0x09, 0x96, 0x24, + 0x83, 0x05, 0xe8, 0xc2, 0x70, 0x95, 0x97, 0xa4, 0xc2, 0xd8, 0x8e, 0xf0, 0x0c, 0x26, 0x71, 0x5d, + 0xc2, 0x11, 0x7a, 0x55, 0xc9, 0x38, 0xc1, 0x09, 0xe3, 0x75, 0x8e, 0xe3, 0xea, 0x7f, 0xc9, 0xc3, + 0x9a, 0x08, 0x21, 0xbe, 0x07, 0x39, 0x3b, 0x0c, 0x26, 0x49, 0x4c, 0xee, 0x12, 0xd5, 0x96, 0x7e, + 0xad, 0x08, 0xb5, 0xe0, 0xfb, 0xb0, 0x66, 0xd9, 0xb6, 0x39, 0x3a, 0x49, 0x1f, 0xda, 0xce, 0x69, + 0x99, 0x7b, 0x2b, 0x46, 0xde, 0x22, 0x75, 0xf3, 0x13, 0x28, 0x21, 0xfd, 0x2c, 0x52, 0xb2, 0xbc, + 0xa8, 0x3b, 0x4b, 0x7d, 0x70, 0x6f, 0xc5, 0x28, 0x5a, 0x52, 0x37, 0xfc, 0x7e, 0xda, 0xfd, 0x9d, + 0x5b, 0x68, 0xe0, 0x9c, 0x32, 0x33, 0xe7, 0x08, 0xff, 0x75, 0xe0, 0xfe, 0xd0, 0x64, 0xc7, 0xce, + 0xab, 0xe7, 0x83, 0x0b, 0xfb, 0xfb, 0xde, 0x8a, 0xc1, 0xf7, 0x2d, 0xb9, 0xdf, 0x7f, 0x24, 0x5d, + 0xd3, 0xc9, 0xaf, 0x3a, 0x2c, 0xe9, 0x19, 0x14, 0x83, 0x89, 0x7f, 0x9a, 0x64, 0x22, 0xb2, 0xd9, + 0xb6, 0x0c, 0x98, 0x2b, 0x2c, 0xb0, 0x25, 0xbb, 0x3a, 0xb1, 0x25, 0x5b, 0xfc, 0x63, 0x28, 0x73, + 0x4f, 0x25, 0xe7, 0x2b, 0x2e, 0x74, 0xed, 0x6c, 0x53, 0xa6, 0xb3, 0xaf, 0xd9, 0x16, 0xdd, 0x90, + 0xed, 0x0c, 0x1d, 0xf5, 0x78, 0xe1, 0xda, 0xd2, 0x8e, 0x32, 0x92, 0x93, 0x06, 0xde, 0x58, 0x83, + 0xf3, 0xb0, 0x36, 0x6c, 0x09, 0x3f, 0x3c, 0xdf, 0x80, 0xe5, 0x9e, 0x09, 0x0b, 0xe3, 0x95, 0xda, + 0xa1, 0xf7, 0x56, 0x0c, 0x66, 0x2d, 0xee, 0xdb, 0x0d, 0xd8, 0x94, 0x55, 0xe2, 0x17, 0x39, 0x67, + 0x51, 0x41, 0x6a, 0x93, 0x66, 0xfb, 0xee, 0xde, 0x8a, 0xb1, 0x61, 0xa5, 0x41, 0xac, 0x05, 0x17, + 0x64, 0x26, 0xe4, 0x8f, 0x16, 0x3d, 0x53, 0x59, 0x18, 0x45, 0x75, 0xaf, 0xde, 0x5b, 0x31, 0x36, + 0xad, 0x79, 0xe0, 0xec, 0x74, 0xfe, 0xaa, 0x01, 0x97, 0x96, 0x8b, 0x04, 0x75, 0x5f, 0xc8, 0xf1, + 0x7d, 0x41, 0x4f, 0xbf, 0x12, 0x94, 0xbe, 0xb1, 0xaf, 0xec, 0x12, 0x3f, 0x84, 0xf5, 0x94, 0x4c, + 0x64, 0x65, 0x28, 0xc8, 0xf7, 0x84, 0x29, 0xce, 0xbf, 0xd1, 0x3d, 0xf8, 0x5c, 0xcb, 0x20, 0xb8, + 0xd5, 0xe9, 0xf5, 0xeb, 0x1d, 0x11, 0x7b, 0xd1, 0xea, 0x88, 0xd8, 0x0b, 0xfd, 0x2f, 0x65, 0xa1, + 0x94, 0x9c, 0x1d, 0x7d, 0x7b, 0x1f, 0x4f, 0xe2, 0x3c, 0xc9, 0xaa, 0xce, 0x93, 0x39, 0x03, 0x86, + 0x3f, 0xfd, 0xcd, 0x5f, 0x8f, 0xda, 0x48, 0x9b, 0x09, 0xd1, 0xe2, 0x4d, 0xe0, 0xfc, 0xd7, 0xbc, + 0x09, 0xac, 0x46, 0x37, 0xaf, 0xa5, 0xa3, 0x9b, 0xe7, 0xde, 0x94, 0x2e, 0xd0, 0x6b, 0xaf, 0xea, + 0x9b, 0xd2, 0xf4, 0xc3, 0x6f, 0xcf, 0x5d, 0xe7, 0x54, 0x84, 0x03, 0x8b, 0x54, 0x7a, 0x4b, 0x85, + 0xd7, 0x6c, 0xa9, 0x5f, 0x47, 0x3c, 0x3f, 0x84, 0xad, 0xd1, 0x49, 0xf2, 0xc6, 0xec, 0xcc, 0x65, + 0x50, 0xa1, 0x2a, 0x2d, 0xc5, 0xe9, 0xff, 0x5f, 0x06, 0x60, 0x76, 0x58, 0xf2, 0x4b, 0xfb, 0x1d, + 0x15, 0xd7, 0x4e, 0xf6, 0x2b, 0x5c, 0x3b, 0xaf, 0x7b, 0x29, 0xe8, 0x4b, 0x28, 0x25, 0xc7, 0x63, + 0xdf, 0x7e, 0xbe, 0x7c, 0xa3, 0x22, 0x7f, 0x4b, 0xfa, 0x60, 0x93, 0xf3, 0xa5, 0x5f, 0xb6, 0x2f, + 0x52, 0xc5, 0x67, 0x5f, 0x53, 0xfc, 0x19, 0x77, 0x84, 0x26, 0x85, 0xff, 0x8a, 0x17, 0x89, 0x3a, + 0x7f, 0x73, 0xa9, 0xf9, 0xab, 0x4f, 0x85, 0x37, 0xf7, 0x97, 0x2f, 0xfa, 0x1b, 0x35, 0xf8, 0x3f, + 0x65, 0xa4, 0xcb, 0x31, 0x79, 0xed, 0xf7, 0x95, 0xca, 0xe7, 0x72, 0xaf, 0xe9, 0x37, 0x29, 0xee, + 0x2b, 0x1d, 0x2a, 0xb9, 0xaf, 0x72, 0xa8, 0xbc, 0x0d, 0x79, 0xbe, 0xed, 0xe4, 0x5f, 0xe5, 0x4c, + 0xe1, 0xf8, 0xd7, 0xbe, 0xc9, 0xaf, 0xeb, 0x42, 0xd9, 0xe6, 0xed, 0xdd, 0x92, 0xf9, 0xca, 0xdf, + 0x13, 0xa0, 0x4b, 0x15, 0xbf, 0x9d, 0xe1, 0x92, 0xf2, 0xdb, 0xf6, 0xc9, 0xaf, 0xcc, 0xa3, 0xf2, + 0x3f, 0x32, 0xb0, 0x9e, 0x3a, 0x19, 0xff, 0x16, 0x95, 0x59, 0x2a, 0x99, 0xb3, 0xff, 0x0b, 0x49, + 0xe6, 0x54, 0x50, 0x69, 0x31, 0x1d, 0x54, 0x8a, 0x92, 0xb1, 0x92, 0xb2, 0x3a, 0x96, 0xd9, 0x27, + 0x99, 0xa5, 0xf6, 0xc9, 0xf5, 0xe4, 0x47, 0xc7, 0x5a, 0xbb, 0x3c, 0x86, 0x73, 0xdd, 0x50, 0x20, + 0xec, 0x53, 0xb8, 0x22, 0xec, 0x7e, 0xde, 0x3f, 0xc1, 0xc8, 0x4c, 0x7e, 0x92, 0x4c, 0xd8, 0xd1, + 0x97, 0x38, 0x01, 0xff, 0xf1, 0x85, 0x51, 0x5d, 0x62, 0xf5, 0x16, 0xac, 0xa7, 0x42, 0x0e, 0x94, + 0x9f, 0x40, 0xcc, 0xa8, 0x3f, 0x81, 0xc8, 0xb6, 0x21, 0x7f, 0x7a, 0xec, 0x84, 0xce, 0x92, 0x87, + 0x43, 0x39, 0x42, 0xff, 0x1e, 0x54, 0xd4, 0xf0, 0x27, 0xf6, 0x2e, 0xe4, 0xdd, 0xd8, 0x19, 0x4b, + 0x8f, 0xc6, 0xa5, 0xc5, 0x08, 0xa9, 0x56, 0xec, 0x8c, 0x0d, 0x4e, 0xa4, 0xff, 0x3c, 0x03, 0xda, + 0x3c, 0x4e, 0xf9, 0x9d, 0xc6, 0xcc, 0x2b, 0x7e, 0xa7, 0x71, 0x35, 0x55, 0xc9, 0x65, 0x3f, 0xb5, + 0x98, 0x3c, 0x5e, 0x98, 0x7b, 0xc5, 0xe3, 0x85, 0xec, 0x0e, 0x14, 0x43, 0x87, 0x7e, 0x04, 0xcf, + 0x5e, 0x72, 0x2f, 0x21, 0xc1, 0xe9, 0xbf, 0x9b, 0x81, 0x82, 0x88, 0xd5, 0x5a, 0xea, 0x62, 0xfa, + 0x0e, 0x14, 0xf8, 0x0f, 0xe2, 0xc9, 0x47, 0x8b, 0x16, 0x02, 0x9f, 0x25, 0x9e, 0x5d, 0xe7, 0x11, + 0x6c, 0x69, 0x97, 0xd3, 0x81, 0x67, 0xf9, 0x06, 0xc1, 0xc5, 0x6f, 0xaa, 0x58, 0x63, 0x71, 0xab, + 0x99, 0x3f, 0x2d, 0x04, 0x04, 0xa2, 0x0b, 0xcc, 0xfa, 0xf7, 0xa1, 0x20, 0x62, 0xc1, 0x96, 0x56, + 0xe5, 0x75, 0x3f, 0x86, 0xb6, 0x0d, 0x30, 0x0b, 0x0e, 0x5b, 0x96, 0x83, 0x7e, 0x0f, 0x8a, 0x32, + 0x1e, 0x0c, 0xe7, 0xdf, 0xac, 0x68, 0x71, 0xe1, 0x45, 0xad, 0x8c, 0x27, 0x1e, 0xe3, 0x6e, 0x07, + 0xc3, 0x13, 0x72, 0x02, 0x3f, 0x00, 0xba, 0xfd, 0xd3, 0x5f, 0x78, 0x83, 0x29, 0xfd, 0x92, 0x7a, + 0x42, 0xc4, 0xee, 0x41, 0x22, 0x5a, 0x5f, 0xe7, 0x0d, 0xd0, 0xeb, 0xf2, 0x9e, 0x18, 0xcd, 0xb2, + 0x47, 0xc2, 0xd9, 0xd9, 0xa6, 0x97, 0xcb, 0x32, 0xea, 0x7b, 0xfa, 0xa9, 0x3a, 0x19, 0x0a, 0x99, + 0x5e, 0x85, 0x8a, 0x1a, 0xc4, 0xa2, 0xd7, 0x61, 0x73, 0xdf, 0x89, 0x2d, 0x94, 0x3f, 0xf2, 0x31, + 0x1b, 0x3e, 0x7f, 0xf1, 0x23, 0x3d, 0x7f, 0xe7, 0xe9, 0x0c, 0x4e, 0xa4, 0xff, 0x3c, 0x07, 0xda, + 0x3c, 0xee, 0xab, 0xee, 0xcc, 0xdd, 0x80, 0x72, 0x40, 0xf3, 0x22, 0xf5, 0xab, 0x39, 0x1c, 0xa4, + 0x44, 0xa8, 0xa7, 0x7e, 0x3a, 0xa1, 0xe8, 0x46, 0x7b, 0xfc, 0xc7, 0x13, 0x2e, 0xf3, 0x0b, 0x52, + 0x5e, 0x30, 0xa4, 0x69, 0x5d, 0xa1, 0xfb, 0x50, 0xed, 0x60, 0x48, 0x57, 0xf1, 0x84, 0x43, 0x81, + 0x47, 0x56, 0x56, 0x8c, 0xa2, 0xf0, 0x22, 0xd0, 0xb9, 0x94, 0x88, 0x5b, 0x8f, 0x23, 0x71, 0xb9, + 0xb1, 0xc8, 0x01, 0xfd, 0x48, 0x3e, 0xf9, 0x3c, 0x14, 0x3f, 0xf1, 0x92, 0xa5, 0x27, 0x9f, 0x1b, + 0x3e, 0xdd, 0xc4, 0xa3, 0x5f, 0x24, 0x1a, 0x8a, 0x5f, 0x8c, 0x12, 0x8f, 0x6e, 0x23, 0xea, 0x16, + 0xff, 0x11, 0x9c, 0xd0, 0x89, 0x22, 0xfe, 0x6c, 0x5f, 0x49, 0xbc, 0x5c, 0x26, 0x80, 0xc9, 0xfb, + 0x80, 0xe2, 0x27, 0x88, 0x90, 0x04, 0xc4, 0xfb, 0x80, 0xfc, 0x07, 0x88, 0x90, 0xe0, 0x0a, 0x14, + 0x7f, 0x1a, 0xf8, 0x0e, 0x39, 0x26, 0xca, 0x54, 0xab, 0x02, 0xa6, 0xf7, 0xad, 0x89, 0xfe, 0x8f, + 0x32, 0xb0, 0x35, 0xdf, 0xab, 0x34, 0x61, 0x2a, 0x50, 0x6c, 0x74, 0xdb, 0x66, 0xa7, 0xbe, 0xdf, + 0xd4, 0x56, 0xd8, 0x06, 0x94, 0xbb, 0x3b, 0x3f, 0x6a, 0x36, 0xfa, 0x1c, 0x90, 0xa1, 0x1b, 0xe9, + 0x3d, 0x73, 0xaf, 0xb5, 0xbb, 0xdb, 0xec, 0x70, 0xeb, 0xa1, 0xbb, 0xf3, 0x23, 0xb3, 0xdd, 0x6d, + 0xf0, 0x5f, 0x2c, 0x91, 0x67, 0xf6, 0x3d, 0x2d, 0x47, 0x27, 0xfa, 0x14, 0xc8, 0x8d, 0xc9, 0x3c, + 0x8f, 0x53, 0x7e, 0xd1, 0x33, 0x1b, 0x9d, 0xbe, 0xb6, 0x86, 0xa9, 0xce, 0x61, 0xbb, 0x4d, 0x29, + 0x0a, 0x48, 0x6c, 0x74, 0xf7, 0x0f, 0x8c, 0x66, 0xaf, 0x67, 0xf6, 0x5a, 0x3f, 0x69, 0x6a, 0x45, + 0x2a, 0xd9, 0x68, 0x3d, 0x6d, 0x75, 0x38, 0xa0, 0xc4, 0x0a, 0x90, 0xdd, 0x6f, 0x75, 0xf8, 0x4d, + 0xfc, 0xfd, 0xfa, 0x67, 0x5a, 0x19, 0x3f, 0x7a, 0x87, 0xfb, 0x5a, 0x45, 0xff, 0x37, 0x59, 0xa9, + 0x1b, 0x53, 0x78, 0xce, 0xd7, 0xd1, 0x07, 0x97, 0x1d, 0x8b, 0x6d, 0x41, 0xde, 0xa3, 0xbb, 0xc5, + 0xe2, 0xc7, 0x58, 0x29, 0xf1, 0x75, 0x7e, 0x60, 0xf2, 0x16, 0xac, 0x27, 0x67, 0xda, 0xca, 0x2b, + 0xcc, 0x15, 0x09, 0x5c, 0x72, 0x78, 0xb0, 0xb6, 0xe4, 0xf0, 0x60, 0xe2, 0xc6, 0x68, 0xf7, 0xa2, + 0x48, 0xe5, 0x13, 0xa5, 0x84, 0x10, 0xfe, 0xd3, 0xae, 0x6f, 0x00, 0x25, 0xcc, 0xa9, 0xef, 0xca, + 0x9f, 0x17, 0x2b, 0x22, 0xe0, 0xd0, 0x77, 0xe3, 0xf9, 0x33, 0xf5, 0xd2, 0xc2, 0x99, 0xba, 0xba, + 0xf7, 0x42, 0x7a, 0xef, 0x4d, 0xff, 0xee, 0x26, 0xff, 0x5d, 0x31, 0xe5, 0x77, 0x37, 0xdf, 0x05, + 0x36, 0x9c, 0x86, 0xf4, 0x4e, 0x96, 0x42, 0x56, 0x21, 0x32, 0x4d, 0x60, 0x92, 0x4d, 0x8f, 0xbd, + 0x0d, 0x1b, 0x73, 0xd4, 0xe4, 0x67, 0x2a, 0x19, 0xd5, 0x34, 0x29, 0xbb, 0x0f, 0x17, 0xc4, 0xd4, + 0x4d, 0xf5, 0xad, 0xb8, 0xb7, 0xc8, 0x51, 0xf5, 0x59, 0x0f, 0xeb, 0xbf, 0x06, 0x45, 0x19, 0x89, + 0xf5, 0xd5, 0x6a, 0xef, 0x92, 0x71, 0xd5, 0xff, 0xce, 0x2a, 0x94, 0x92, 0xb8, 0xac, 0xaf, 0x35, + 0x3b, 0xe8, 0xb9, 0xf9, 0xe8, 0x44, 0x95, 0x20, 0x45, 0x04, 0xc8, 0x91, 0x12, 0xd7, 0x83, 0xa6, + 0xa1, 0x2b, 0x55, 0x37, 0x0e, 0x39, 0x0c, 0x5d, 0x7a, 0x70, 0xd1, 0xf5, 0x95, 0x1b, 0x86, 0x25, + 0xa3, 0x88, 0x00, 0x5a, 0x5d, 0x57, 0x80, 0xbe, 0x89, 0x53, 0xfe, 0x14, 0xa9, 0xeb, 0x9f, 0x20, + 0xdf, 0x2b, 0x7e, 0x8a, 0x94, 0x1e, 0xcb, 0xe7, 0x41, 0x21, 0xfc, 0x28, 0x5c, 0xfe, 0xb4, 0xd3, + 0x1b, 0x50, 0x9a, 0x26, 0xbf, 0x0d, 0x26, 0x66, 0xc4, 0x54, 0xfe, 0x32, 0x58, 0x7a, 0x54, 0x4b, + 0xf3, 0xa3, 0x3a, 0x3f, 0xa7, 0x61, 0x61, 0x4e, 0xeb, 0x31, 0x14, 0x44, 0x6c, 0xda, 0x57, 0x77, + 0xf8, 0x57, 0x76, 0x95, 0x06, 0x59, 0xcb, 0x93, 0xd7, 0x1a, 0xf1, 0x73, 0xae, 0x62, 0xb9, 0xb9, + 0x8a, 0xe9, 0x7f, 0x75, 0x15, 0x60, 0x16, 0xe3, 0xc6, 0xde, 0x9b, 0x8b, 0xa7, 0xcd, 0x2c, 0xec, + 0xea, 0x73, 0x61, 0xb4, 0x73, 0xaf, 0xbd, 0xac, 0x7e, 0x8d, 0xd7, 0x5e, 0x1e, 0xc2, 0x7a, 0x14, + 0x0e, 0x5f, 0xeb, 0x02, 0x2f, 0x47, 0xe1, 0x30, 0xf1, 0x80, 0x3f, 0x00, 0x4c, 0xd2, 0xcb, 0x71, + 0x33, 0x93, 0x75, 0x41, 0x29, 0x29, 0x45, 0xe1, 0xb0, 0x3b, 0xf8, 0x62, 0x97, 0xdf, 0xc9, 0xb2, + 0xa3, 0xd8, 0x5c, 0x26, 0x25, 0x36, 0xec, 0x28, 0xde, 0x55, 0x05, 0xc5, 0x6d, 0xa8, 0x22, 0xed, + 0x82, 0xb0, 0xa8, 0xd8, 0xd1, 0xec, 0xc8, 0x43, 0xef, 0x88, 0x83, 0xd4, 0x39, 0xe7, 0x2a, 0xfb, + 0x58, 0x58, 0xe4, 0x8a, 0x8a, 0x50, 0x5b, 0xe6, 0x8b, 0xe5, 0x4f, 0xd3, 0x24, 0xa4, 0xf7, 0x6e, + 0x42, 0x45, 0xfd, 0xe9, 0x44, 0xba, 0xde, 0x11, 0xf8, 0x0e, 0x7f, 0x5c, 0xbf, 0xfd, 0xd3, 0x0f, + 0xb5, 0xcc, 0x3d, 0x1d, 0xca, 0xca, 0x4f, 0x5b, 0x20, 0xc5, 0x9e, 0x15, 0x1d, 0x8b, 0x87, 0xd6, + 0x2d, 0xff, 0xc8, 0xd1, 0x32, 0xf7, 0xee, 0xa0, 0xda, 0xab, 0xfe, 0xb0, 0x04, 0xc0, 0x5a, 0x27, + 0x08, 0xc7, 0x96, 0x27, 0xe8, 0x9c, 0x69, 0x84, 0x74, 0x0f, 0xe0, 0xe2, 0xd2, 0x9f, 0xc9, 0xa0, + 0x3b, 0x42, 0xee, 0x78, 0xe2, 0x39, 0xfc, 0x9a, 0xcb, 0xde, 0xf9, 0x20, 0x74, 0x6d, 0x2d, 0x73, + 0xef, 0xb1, 0xbc, 0x33, 0x2f, 0xcb, 0x6e, 0x77, 0xeb, 0xbb, 0x7c, 0x87, 0x4a, 0xde, 0x6e, 0xe9, + 0xef, 0xf0, 0x47, 0xd4, 0x8d, 0x66, 0xef, 0xb0, 0xdd, 0x17, 0xef, 0xc4, 0xdc, 0xfb, 0x21, 0xd4, + 0x5e, 0x75, 0xdf, 0x03, 0x6b, 0xd4, 0xd8, 0xab, 0xd3, 0x9d, 0x1a, 0xdc, 0x91, 0xba, 0x26, 0x4f, + 0x65, 0xf8, 0x95, 0xa4, 0x76, 0x93, 0x82, 0x22, 0xef, 0xfd, 0x2c, 0xa3, 0xe8, 0x61, 0x32, 0x66, + 0x3f, 0x01, 0x88, 0x6e, 0x52, 0x41, 0x86, 0x63, 0xd9, 0x5a, 0x86, 0x5d, 0x02, 0x96, 0x02, 0xb5, + 0x83, 0xa1, 0xe5, 0x69, 0xab, 0x14, 0xfe, 0x28, 0xe1, 0x74, 0x2b, 0x4b, 0xcb, 0xb2, 0x37, 0xe1, + 0x4a, 0x02, 0x6b, 0x07, 0xa7, 0x07, 0xa1, 0x1b, 0x84, 0x6e, 0x7c, 0xce, 0xd1, 0xb9, 0x7b, 0xff, + 0x87, 0x38, 0x97, 0x4c, 0x8d, 0x1f, 0x16, 0x50, 0xb7, 0xed, 0x19, 0x8c, 0x24, 0x86, 0xb6, 0xc2, + 0x2e, 0xc3, 0x05, 0x12, 0x97, 0x73, 0x88, 0x0c, 0x7b, 0x03, 0x2e, 0x4b, 0x63, 0x71, 0x1e, 0xb9, + 0x8a, 0x48, 0xc3, 0xa1, 0xd0, 0xb9, 0x05, 0x64, 0x76, 0xe7, 0x07, 0x7f, 0xfa, 0x8b, 0xeb, 0x99, + 0x7f, 0xfa, 0x8b, 0xeb, 0x99, 0x7f, 0xfb, 0x8b, 0xeb, 0x2b, 0x3f, 0xff, 0x77, 0xd7, 0x33, 0x3f, + 0x79, 0xef, 0xc8, 0x8d, 0x8f, 0xa7, 0x83, 0xfb, 0xc3, 0x60, 0xfc, 0x60, 0x6c, 0xc5, 0xa1, 0x7b, + 0xc6, 0xe5, 0xb6, 0x4c, 0xf8, 0xce, 0x83, 0xc9, 0xc9, 0xd1, 0x83, 0xc9, 0xe0, 0x01, 0x4e, 0xbf, + 0xc1, 0xda, 0x24, 0x0c, 0xe2, 0xe0, 0xd1, 0xff, 0x0c, 0x00, 0x00, 0xff, 0xff, 0x0d, 0x87, 0xdd, + 0x89, 0xc4, 0x82, 0x00, 0x00, } func (m *Type) Marshal() (dAtA []byte, err error) { @@ -22912,6 +23006,18 @@ func (m *AlterTable) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.AlterPartition != nil { + { + size, err := m.AlterPartition.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintPlan(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x7a + } if len(m.AffectedCols) > 0 { for iNdEx := len(m.AffectedCols) - 1; iNdEx >= 0; iNdEx-- { i -= len(m.AffectedCols[iNdEx]) @@ -23372,20 +23478,20 @@ func (m *DropTable) MarshalToSizedBuffer(dAtA []byte) (int, error) { copy(dAtA[i:], m.XXX_unrecognized) } if len(m.FkChildTblsReferToMe) > 0 { - dAtA190 := make([]byte, len(m.FkChildTblsReferToMe)*10) - var j189 int + dAtA191 := make([]byte, len(m.FkChildTblsReferToMe)*10) + var j190 int for _, num := range m.FkChildTblsReferToMe { for num >= 1<<7 { - dAtA190[j189] = uint8(uint64(num)&0x7f | 0x80) + dAtA191[j190] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j189++ + j190++ } - dAtA190[j189] = uint8(num) - j189++ + dAtA191[j190] = uint8(num) + j190++ } - i -= j189 - copy(dAtA[i:], dAtA190[:j189]) - i = encodeVarintPlan(dAtA, i, uint64(j189)) + i -= j190 + copy(dAtA[i:], dAtA191[:j190]) + i = encodeVarintPlan(dAtA, i, uint64(j190)) i-- dAtA[i] = 0x62 } @@ -23421,20 +23527,20 @@ func (m *DropTable) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x48 } if len(m.ForeignTbl) > 0 { - dAtA193 := make([]byte, len(m.ForeignTbl)*10) - var j192 int + dAtA194 := make([]byte, len(m.ForeignTbl)*10) + var j193 int for _, num := range m.ForeignTbl { for num >= 1<<7 { - dAtA193[j192] = uint8(uint64(num)&0x7f | 0x80) + dAtA194[j193] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j192++ + j193++ } - dAtA193[j192] = uint8(num) - j192++ + dAtA194[j193] = uint8(num) + j193++ } - i -= j192 - copy(dAtA[i:], dAtA193[:j192]) - i = encodeVarintPlan(dAtA, i, uint64(j192)) + i -= j193 + copy(dAtA[i:], dAtA194[:j193]) + i = encodeVarintPlan(dAtA, i, uint64(j193)) i-- dAtA[i] = 0x3a } @@ -23987,20 +24093,20 @@ func (m *TruncateTable) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x40 } if len(m.ForeignTbl) > 0 { - dAtA202 := make([]byte, len(m.ForeignTbl)*10) - var j201 int + dAtA203 := make([]byte, len(m.ForeignTbl)*10) + var j202 int for _, num := range m.ForeignTbl { for num >= 1<<7 { - dAtA202[j201] = uint8(uint64(num)&0x7f | 0x80) + dAtA203[j202] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j201++ + j202++ } - dAtA202[j201] = uint8(num) - j201++ + dAtA203[j202] = uint8(num) + j202++ } - i -= j201 - copy(dAtA[i:], dAtA202[:j201]) - i = encodeVarintPlan(dAtA, i, uint64(j201)) + i -= j202 + copy(dAtA[i:], dAtA203[:j202]) + i = encodeVarintPlan(dAtA, i, uint64(j202)) i-- dAtA[i] = 0x3a } @@ -24077,20 +24183,20 @@ func (m *ClusterTable) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x18 } if len(m.AccountIDs) > 0 { - dAtA205 := make([]byte, len(m.AccountIDs)*10) - var j204 int + dAtA206 := make([]byte, len(m.AccountIDs)*10) + var j205 int for _, num := range m.AccountIDs { for num >= 1<<7 { - dAtA205[j204] = uint8(uint64(num)&0x7f | 0x80) + dAtA206[j205] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j204++ + j205++ } - dAtA205[j204] = uint8(num) - j204++ + dAtA206[j205] = uint8(num) + j205++ } - i -= j204 - copy(dAtA[i:], dAtA205[:j204]) - i = encodeVarintPlan(dAtA, i, uint64(j204)) + i -= j205 + copy(dAtA[i:], dAtA206[:j205]) + i = encodeVarintPlan(dAtA, i, uint64(j205)) i-- dAtA[i] = 0x12 } @@ -24302,21 +24408,21 @@ func (m *Prepare) MarshalToSizedBuffer(dAtA []byte) (int, error) { copy(dAtA[i:], m.XXX_unrecognized) } if len(m.ParamTypes) > 0 { - dAtA209 := make([]byte, len(m.ParamTypes)*10) - var j208 int + dAtA210 := make([]byte, len(m.ParamTypes)*10) + var j209 int for _, num1 := range m.ParamTypes { num := uint64(num1) for num >= 1<<7 { - dAtA209[j208] = uint8(uint64(num)&0x7f | 0x80) + dAtA210[j209] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j208++ + j209++ } - dAtA209[j208] = uint8(num) - j208++ + dAtA210[j209] = uint8(num) + j209++ } - i -= j208 - copy(dAtA[i:], dAtA209[:j208]) - i = encodeVarintPlan(dAtA, i, uint64(j208)) + i -= j209 + copy(dAtA[i:], dAtA210[:j209]) + i = encodeVarintPlan(dAtA, i, uint64(j209)) i-- dAtA[i] = 0x22 } @@ -24463,21 +24569,21 @@ func (m *OtherDCL) MarshalToSizedBuffer(dAtA []byte) (int, error) { copy(dAtA[i:], m.XXX_unrecognized) } if len(m.ParamTypes) > 0 { - dAtA212 := make([]byte, len(m.ParamTypes)*10) - var j211 int + dAtA213 := make([]byte, len(m.ParamTypes)*10) + var j212 int for _, num1 := range m.ParamTypes { num := uint64(num1) for num >= 1<<7 { - dAtA212[j211] = uint8(uint64(num)&0x7f | 0x80) + dAtA213[j212] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j211++ + j212++ } - dAtA212[j211] = uint8(num) - j211++ + dAtA213[j212] = uint8(num) + j212++ } - i -= j211 - copy(dAtA[i:], dAtA212[:j211]) - i = encodeVarintPlan(dAtA, i, uint64(j211)) + i -= j212 + copy(dAtA[i:], dAtA213[:j212]) + i = encodeVarintPlan(dAtA, i, uint64(j212)) i-- dAtA[i] = 0xa } @@ -25147,6 +25253,38 @@ func (m *CloneTable) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *AlterPartitionOption) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *AlterPartitionOption) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *AlterPartitionOption) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.AlterType != 0 { + i = encodeVarintPlan(dAtA, i, uint64(m.AlterType)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func encodeVarintPlan(dAtA []byte, offset int, v uint64) int { offset -= sovPlan(v) base := offset @@ -29315,6 +29453,10 @@ func (m *AlterTable) ProtoSize() (n int) { n += 1 + l + sovPlan(uint64(l)) } } + if m.AlterPartition != nil { + l = m.AlterPartition.ProtoSize() + n += 1 + l + sovPlan(uint64(l)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -30280,6 +30422,21 @@ func (m *CloneTable) ProtoSize() (n int) { return n } +func (m *AlterPartitionOption) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.AlterType != 0 { + n += 1 + sovPlan(uint64(m.AlterType)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + func sovPlan(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } @@ -53418,6 +53575,42 @@ func (m *AlterTable) Unmarshal(dAtA []byte) error { } m.AffectedCols = append(m.AffectedCols, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex + case 15: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AlterPartition", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPlan + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthPlan + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthPlan + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.AlterPartition == nil { + m.AlterPartition = &AlterPartitionOption{} + } + if err := m.AlterPartition.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipPlan(dAtA[iNdEx:]) @@ -58930,6 +59123,76 @@ func (m *CloneTable) Unmarshal(dAtA []byte) error { } return nil } +func (m *AlterPartitionOption) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPlan + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: AlterPartitionOption: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: AlterPartitionOption: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AlterType", wireType) + } + m.AlterType = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPlan + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.AlterType |= AlterPartitionType(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipPlan(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthPlan + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipPlan(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 diff --git a/pkg/sql/compile/alter.go b/pkg/sql/compile/alter.go index 785d2815556c1..44efaff1fe99c 100644 --- a/pkg/sql/compile/alter.go +++ b/pkg/sql/compile/alter.go @@ -20,15 +20,14 @@ import ( "slices" "github.com/matrixorigin/matrixone/pkg/catalog" - "github.com/matrixorigin/matrixone/pkg/common/reuse" - "github.com/matrixorigin/matrixone/pkg/logutil" - "github.com/matrixorigin/matrixone/pkg/sql/colexec/table_clone" - "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/common/reuse" "github.com/matrixorigin/matrixone/pkg/defines" + "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/pb/api" "github.com/matrixorigin/matrixone/pkg/pb/lock" "github.com/matrixorigin/matrixone/pkg/pb/plan" + "github.com/matrixorigin/matrixone/pkg/sql/colexec/table_clone" "github.com/matrixorigin/matrixone/pkg/sql/features" "github.com/matrixorigin/matrixone/pkg/sql/parsers" "github.com/matrixorigin/matrixone/pkg/sql/parsers/dialect" @@ -394,52 +393,143 @@ func (s *Scope) AlterTable(c *Compile) (err error) { return s.doAlterTable(c) } - switch qry.AlgorithmType { - case plan.AlterTable_COPY: - return s.doAlterTable(c) - default: - // alter primary table - if err := s.doAlterTable(c); err != nil { - return err + if qry.AlterPartition == nil { + switch qry.AlgorithmType { + case plan.AlterTable_COPY: + return s.doAlterTable(c) + default: + // alter primary table + if err := s.doAlterTable(c); err != nil { + return err + } + + // alter all partition tables + if qry.RawSQL == "" { + for _, ac := range qry.Actions { + if _, ok := ac.Action.(*plan.AlterTable_Action_AlterName); ok { + value := ac.Action.(*plan.AlterTable_Action_AlterName) + return ps.Rename( + c.proc.Ctx, + qry.TableDef.TblId, + value.AlterName.OldName, + value.AlterName.NewName, + c.proc.GetTxnOperator(), + ) + } + } + + panic("missing RawSQL for alter partition tables") + } + + metadata, err := ps.GetPartitionMetadata( + c.proc.Ctx, + qry.TableDef.TblId, + c.proc.Base.TxnOperator, + ) + if err != nil { + return err + } + + st, _ := parsers.ParseOne( + c.proc.Ctx, + dialect.MYSQL, + qry.RawSQL, + c.getLower(), + ) + stmt := st.(*tree.AlterTable) + table := stmt.Table + stmt.PartitionOption = nil + for _, p := range metadata.Partitions { + stmt.Table = tree.NewTableName( + tree.Identifier(p.PartitionTableName), + table.ObjectNamePrefix, + table.AtTsExpr, + ) + sql := tree.StringWithOpts( + stmt, + dialect.MYSQL, + tree.WithQuoteIdentifier(), + tree.WithSingleQuoteString(), + ) + if err := c.runSql(sql); err != nil { + return err + } + } + return nil } + } + + switch qry.AlterPartition.AlterType { + case plan.AlterPartitionType_AddPartitionTables: + stmt, _ := parsers.ParseOne( + c.proc.Ctx, + dialect.MYSQL, + qry.RawSQL, + c.getLower(), + ) - // alter all partition tables - metadata, err := ps.GetPartitionMetadata( + return ps.AddPartitions( c.proc.Ctx, qry.TableDef.TblId, - c.proc.Base.TxnOperator, + stmt.(*tree.AlterTable).PartitionOption.(*tree.AlterPartitionAddPartitionClause).Partitions, + c.proc.GetTxnOperator(), ) - if err != nil { - return err + case plan.AlterPartitionType_DropPartitionTables: + stmt, _ := parsers.ParseOne( + c.proc.Ctx, + dialect.MYSQL, + qry.RawSQL, + c.getLower(), + ) + + names := stmt.(*tree.AlterTable).PartitionOption.(*tree.AlterPartitionDropPartitionClause).PartitionNames + partitions := make([]string, 0, len(names)) + for _, p := range names { + partitions = append(partitions, p.String()) } - st, _ := parsers.ParseOne( + return ps.DropPartitions( + c.proc.Ctx, + qry.TableDef.TblId, + partitions, + c.proc.GetTxnOperator(), + ) + case plan.AlterPartitionType_TruncatePartitionTables: + stmt, _ := parsers.ParseOne( c.proc.Ctx, dialect.MYSQL, qry.RawSQL, c.getLower(), ) - stmt := st.(*tree.AlterTable) - table := stmt.Table - stmt.PartitionOption = nil - for _, p := range metadata.Partitions { - stmt.Table = tree.NewTableName( - tree.Identifier(p.PartitionTableName), - table.ObjectNamePrefix, - table.AtTsExpr, - ) - sql := tree.StringWithOpts( - stmt, - dialect.MYSQL, - tree.WithQuoteIdentifier(), - tree.WithSingleQuoteString(), - ) - if err := c.runSql(sql); err != nil { - return err - } + var partitions []string + names := stmt.(*tree.AlterTable).PartitionOption.(*tree.AlterPartitionTruncatePartitionClause).PartitionNames + for _, p := range names { + partitions = append(partitions, p.String()) } - return nil + + return ps.TruncatePartitions( + c.proc.Ctx, + qry.TableDef.TblId, + partitions, + c.proc.GetTxnOperator(), + ) + case plan.AlterPartitionType_RedefinePartitionTables: + stmt, _ := parsers.ParseOne( + c.proc.Ctx, + dialect.MYSQL, + qry.RawSQL, + c.getLower(), + ) + newOptions := stmt.(*tree.AlterTable).PartitionOption.(*tree.AlterPartitionRedefinePartitionClause).PartitionOption + + return ps.Redefine( + c.proc.Ctx, + qry.TableDef.TblId, + newOptions, + c.proc.GetTxnOperator(), + ) } + return moerr.NewInternalError(c.proc.Ctx, "unsupported alter partition type") } func (s *Scope) doAlterTable(c *Compile) error { diff --git a/pkg/sql/parsers/tree/alter.go b/pkg/sql/parsers/tree/alter.go index 422c99a7bc90e..c54bab6b1baad 100644 --- a/pkg/sql/parsers/tree/alter.go +++ b/pkg/sql/parsers/tree/alter.go @@ -495,7 +495,7 @@ func NewAlterTable(table *TableName) *AlterTable { } func (node *AlterTable) Free() { - reuse.Free[AlterTable](node, nil) + reuse.Free(node, nil) } func (node *AlterTable) Format(ctx *FmtCtx) { @@ -1471,7 +1471,7 @@ func NewAlterPartitionRedefinePartitionClause(typ AlterPartitionOptionType, part } func (node *AlterPartitionRedefinePartitionClause) Free() { - reuse.Free[AlterPartitionRedefinePartitionClause](node, nil) + reuse.Free(node, nil) } func (node *AlterPartitionRedefinePartitionClause) Format(ctx *FmtCtx) { @@ -1504,7 +1504,7 @@ func NewAlterPartitionAddPartitionClause(typ AlterPartitionOptionType, partition } func (node *AlterPartitionAddPartitionClause) Free() { - reuse.Free[AlterPartitionAddPartitionClause](node, nil) + reuse.Free(node, nil) } func (node *AlterPartitionAddPartitionClause) Format(ctx *FmtCtx) { @@ -1550,7 +1550,7 @@ func NewAlterPartitionDropPartitionClause(typ AlterPartitionOptionType, partitio } func (node *AlterPartitionDropPartitionClause) Free() { - reuse.Free[AlterPartitionDropPartitionClause](node, nil) + reuse.Free(node, nil) } func (node *AlterPartitionDropPartitionClause) Format(ctx *FmtCtx) { @@ -1581,7 +1581,7 @@ func NewAlterPartitionTruncatePartitionClause(typ AlterPartitionOptionType, part } func (node *AlterPartitionTruncatePartitionClause) Free() { - reuse.Free[AlterPartitionTruncatePartitionClause](node, nil) + reuse.Free(node, nil) } func (node *AlterPartitionTruncatePartitionClause) Format(ctx *FmtCtx) { diff --git a/pkg/sql/plan/build_ddl.go b/pkg/sql/plan/build_ddl.go index d2ef1eef179ae..d766caa3f671a 100644 --- a/pkg/sql/plan/build_ddl.go +++ b/pkg/sql/plan/build_ddl.go @@ -713,11 +713,7 @@ func buildCreateTable( tblName := formatStr(string(oldTable.ObjectName)) dbName := formatStr(string(oldTable.SchemaName)) - var snapshot *Snapshot - snapshot = ctx.GetSnapshot() - if snapshot == nil { - snapshot = &Snapshot{TS: ×tamp.Timestamp{}} - } + snapshot := ctx.GetSnapshot() if dbName, err = databaseIsValid(getSuitableDBName(dbName, ""), ctx, snapshot); err != nil { return nil, err @@ -4190,12 +4186,24 @@ func buildAlterTableInplace(stmt *tree.AlterTable, ctx CompilerContext) (*Plan, } if stmt.PartitionOption != nil { - // TODO: reimplement partition - return nil, moerr.NewNotSupportedf( - ctx.GetContext(), - unsupportedErrFmt, - formatTreeNode(stmt.PartitionOption), - ) + alterTable.AlterPartition = &plan.AlterPartitionOption{} + + switch stmt.PartitionOption.(type) { + case *tree.AlterPartitionAddPartitionClause: + alterTable.AlterPartition.AlterType = plan.AlterPartitionType_AddPartitionTables + case *tree.AlterPartitionDropPartitionClause: + alterTable.AlterPartition.AlterType = plan.AlterPartitionType_DropPartitionTables + case *tree.AlterPartitionTruncatePartitionClause: + alterTable.AlterPartition.AlterType = plan.AlterPartitionType_TruncatePartitionTables + case *tree.AlterPartitionRedefinePartitionClause: + alterTable.AlterPartition.AlterType = plan.AlterPartitionType_RedefinePartitionTables + default: + return nil, moerr.NewNotSupportedf( + ctx.GetContext(), + unsupportedErrFmt, + formatTreeNode(stmt.PartitionOption), + ) + } } // check Constraint Name (include index/ unique) diff --git a/pkg/sql/plan/build_show_util.go b/pkg/sql/plan/build_show_util.go index 5044cd370c6c7..602a6b4104455 100644 --- a/pkg/sql/plan/build_show_util.go +++ b/pkg/sql/plan/build_show_util.go @@ -24,6 +24,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/defines" "github.com/matrixorigin/matrixone/pkg/pb/partition" "github.com/matrixorigin/matrixone/pkg/pb/plan" "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" @@ -418,11 +419,16 @@ func ConstructCreateTableSQL( partitionBy := " partition by " txn := ctx.GetProcess().GetTxnOperator() + newCtx := ctx.GetContext() if snapshot != nil && snapshot.TS != nil { txn = txn.CloneSnapshotOp(*snapshot.TS) + + if snapshot.Tenant != nil { + newCtx = defines.AttachAccountId(newCtx, snapshot.Tenant.TenantID) + } } - meta, err := ps.GetPartitionMetadata(ctx.GetProcess().Ctx, tableDef.GetTblId(), txn) + meta, err := ps.GetPartitionMetadata(newCtx, tableDef.GetTblId(), txn) if err != nil { return "", nil, err } diff --git a/pkg/tests/partition/alter_parittion_test.go b/pkg/tests/partition/alter_parittion_test.go new file mode 100644 index 0000000000000..b95c85ff27aec --- /dev/null +++ b/pkg/tests/partition/alter_parittion_test.go @@ -0,0 +1,342 @@ +// Copyright 2021 - 2025 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package partition + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/matrixorigin/matrixone/pkg/cnservice" + "github.com/matrixorigin/matrixone/pkg/embed" + "github.com/matrixorigin/matrixone/pkg/pb/partition" + "github.com/matrixorigin/matrixone/pkg/tests/testutils" + "github.com/matrixorigin/matrixone/pkg/util/executor" + "github.com/stretchr/testify/require" +) + +// TestAlterTableDropPartition tests the ALTER TABLE DROP PARTITION functionality +func TestAlterTableDropPartition(t *testing.T) { + runPartitionTableCreateAndDeleteTestsWithAware( + t, + func(c embed.Cluster) int32 { return 0 }, + "create table %s (c int comment 'abc', b int) partition by list (c) (partition p0 values in (1, 2), partition p1 values in (3, 4), partition p2 values in (5, 6))", + partition.PartitionMethod_List, + func(idx int, p partition.Partition) { + // Validate partition properties + require.NotEqual(t, uint64(0), p.PartitionID) + require.Equal(t, uint32(idx), p.Position) + }, + func(db string, table string, cn embed.ServiceOperator, pm partition.PartitionMetadata) { + // Insert some test data + testutils.ExecSQLWithReadResult( + t, + db, + cn, + func(i int, s string, r executor.Result) { + // Verify insert was successful + }, + fmt.Sprintf("insert into %s values (1, 10), (2, 20), (3, 30), (4, 40), (5, 50)", table), + ) + + // Verify initial partition count + initialMetadata := getMetadata(t, 0, db, table, cn) + require.Equal(t, 3, len(initialMetadata.Partitions)) + + // Drop one partition (p0) + testutils.ExecSQLWithReadResult( + t, + db, + cn, + func(i int, s string, r executor.Result) { + // Verify drop partition was successful + }, + fmt.Sprintf("alter table %s drop partition p0", table), + ) + + // Verify partition was dropped + cs := cn.RawService().(cnservice.Service) + exec := cs.GetSQLExecutor() + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*20) + defer cancel() + exec.ExecTxn( + ctx, + func(txn executor.TxnExecutor) error { + metadata := getMetadata(t, 0, db, table, cn) + // Should have 2 partitions remaining + require.Equal(t, 2, len(metadata.Partitions)) + + // Verify the remaining partitions are p1 and p2 + partitionNames := make(map[string]bool) + for _, p := range metadata.Partitions { + partitionNames[p.Name] = true + } + require.True(t, partitionNames["p1"]) + require.True(t, partitionNames["p2"]) + require.False(t, partitionNames["p0"]) + + return nil + }, + executor.Options{}, + ) + }, + func(cn embed.ServiceOperator, pm partition.PartitionMetadata) { + // Cleanup verification + }, + ) +} + +// TestAlterTableTruncatePartition tests the ALTER TABLE TRUNCATE PARTITION functionality +func TestAlterTableTruncatePartition(t *testing.T) { + runPartitionTableCreateAndDeleteTestsWithAware( + t, + func(c embed.Cluster) int32 { return 0 }, + "create table %s (c int comment 'abc', b int) partition by list (c) (partition p0 values in (1, 2), partition p1 values in (3, 4), partition p2 values in (5, 6))", + partition.PartitionMethod_List, + func(idx int, p partition.Partition) { + // Validate partition properties + require.NotEqual(t, uint64(0), p.PartitionID) + require.Equal(t, uint32(idx), p.Position) + }, + func(db string, table string, cn embed.ServiceOperator, pm partition.PartitionMetadata) { + // Insert some test data + testutils.ExecSQLWithReadResult( + t, + db, + cn, + func(i int, s string, r executor.Result) { + // Verify insert was successful + }, + fmt.Sprintf("insert into %s values (1, 10), (2, 20), (3, 30), (4, 40), (5, 50)", table), + ) + + // Verify data was inserted + testutils.ExecSQLWithReadResult( + t, + db, + cn, + func(i int, s string, r executor.Result) { + // Should have 5 rows + }, + fmt.Sprintf("select count(*) from %s", table), + ) + + // Truncate one partition (p0) + testutils.ExecSQLWithReadResult( + t, + db, + cn, + func(i int, s string, r executor.Result) { + // Verify truncate partition was successful + }, + fmt.Sprintf("alter table %s truncate partition p0", table), + ) + + // Verify partition was truncated but table structure remains + cs := cn.RawService().(cnservice.Service) + exec := cs.GetSQLExecutor() + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*20) + defer cancel() + exec.ExecTxn( + ctx, + func(txn executor.TxnExecutor) error { + metadata := getMetadata(t, 0, db, table, cn) + // Should still have 3 partitions + require.Equal(t, 3, len(metadata.Partitions)) + + // Verify partition structure is intact + for _, p := range metadata.Partitions { + require.NotEqual(t, uint64(0), p.PartitionID) + } + + return nil + }, + executor.Options{}, + ) + + // Verify some data remains (data in other partitions) + testutils.ExecSQLWithReadResult( + t, + db, + cn, + func(i int, s string, r executor.Result) { + // Some data should remain in other partitions + }, + fmt.Sprintf("select count(*) from %s", table), + ) + }, + func(cn embed.ServiceOperator, pm partition.PartitionMetadata) { + // Cleanup verification + }, + ) +} + +// TestAlterTableAddPartition tests the ALTER TABLE ADD PARTITION functionality +func TestAlterTableAddPartition(t *testing.T) { + runPartitionTableCreateAndDeleteTestsWithAware( + t, + func(c embed.Cluster) int32 { return 0 }, + "create table %s (c int comment 'abc', b int) partition by list (c) (partition p0 values in (1, 2), partition p1 values in (3, 4))", + partition.PartitionMethod_List, + func(idx int, p partition.Partition) { + // Validate partition properties + require.NotEqual(t, uint64(0), p.PartitionID) + require.Equal(t, uint32(idx), p.Position) + }, + func(db string, table string, cn embed.ServiceOperator, pm partition.PartitionMetadata) { + // Insert some test data + testutils.ExecSQLWithReadResult( + t, + db, + cn, + func(i int, s string, r executor.Result) { + // Verify insert was successful + }, + fmt.Sprintf("insert into %s values (1, 10), (2, 20), (3, 30), (4, 40)", table), + ) + + // Verify initial partition count + initialMetadata := getMetadata(t, 0, db, table, cn) + require.Equal(t, 2, len(initialMetadata.Partitions)) + + // Add a new partition (p2) + testutils.ExecSQLWithReadResult( + t, + db, + cn, + func(i int, s string, r executor.Result) { + // Verify add partition was successful + }, + fmt.Sprintf("alter table %s add partition (partition p2 values in (5, 6))", table), + ) + + // Verify partition was added + cs := cn.RawService().(cnservice.Service) + exec := cs.GetSQLExecutor() + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*20) + defer cancel() + exec.ExecTxn( + ctx, + func(txn executor.TxnExecutor) error { + metadata := getMetadata(t, 0, db, table, cn) + // Should have 3 partitions now + require.Equal(t, 3, len(metadata.Partitions)) + + // Verify all partitions exist + partitionNames := make(map[string]bool) + for _, p := range metadata.Partitions { + partitionNames[p.Name] = true + require.NotEqual(t, uint64(0), p.PartitionID) + } + require.True(t, partitionNames["p0"]) + require.True(t, partitionNames["p1"]) + require.True(t, partitionNames["p2"]) + + return nil + }, + executor.Options{}, + ) + }, + func(cn embed.ServiceOperator, pm partition.PartitionMetadata) { + // Cleanup verification + }, + ) +} + +// TestAlterTablePartitionBy tests the ALTER TABLE PARTITION BY functionality +func TestAlterTablePartitionBy(t *testing.T) { + runPartitionTableCreateAndDeleteTestsWithAware( + t, + func(c embed.Cluster) int32 { return 0 }, + "create table %s (c int comment 'abc', b int) partition by hash(c) partitions 2", + partition.PartitionMethod_Hash, + func(idx int, p partition.Partition) { + // Validate initial partition properties + require.NotEqual(t, uint64(0), p.PartitionID) + require.Equal(t, uint32(idx), p.Position) + }, + func(db string, table string, cn embed.ServiceOperator, pm partition.PartitionMetadata) { + // Insert some test data + testutils.ExecSQLWithReadResult( + t, + db, + cn, + func(i int, s string, r executor.Result) { + // Verify insert was successful + }, + fmt.Sprintf("insert into %s values (1, 10), (2, 20), (3, 30), (4, 40)", table), + ) + + // Verify initial partition count + initialMetadata := getMetadata(t, 0, db, table, cn) + require.Equal(t, 2, len(initialMetadata.Partitions)) + require.Equal(t, partition.PartitionMethod_Hash, initialMetadata.Method) + + // Change partition method to hash with different column and partition count + testutils.ExecSQLWithReadResult( + t, + db, + cn, + func(i int, s string, r executor.Result) { + // Verify alter partition by was successful + }, + fmt.Sprintf("alter table %s partition by hash(b) partitions 3", table), + ) + + // Verify partition structure changed + cs := cn.RawService().(cnservice.Service) + exec := cs.GetSQLExecutor() + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*20) + defer cancel() + exec.ExecTxn( + ctx, + func(txn executor.TxnExecutor) error { + metadata := getMetadata(t, 0, db, table, cn) + // Should now have 3 partitions + require.Equal(t, 3, len(metadata.Partitions)) + require.Equal(t, partition.PartitionMethod_Hash, metadata.Method) + + // Verify all partitions have valid IDs + for _, p := range metadata.Partitions { + require.NotEqual(t, uint64(0), p.PartitionID) + require.Equal(t, metadata.TableID, p.PrimaryTableID) + } + + return nil + }, + executor.Options{}, + ) + + // Verify data is still accessible + testutils.ExecSQLWithReadResult( + t, + db, + cn, + func(i int, s string, r executor.Result) { + // Should still have 4 rows + }, + fmt.Sprintf("select count(*) from %s", table), + ) + }, + func(cn embed.ServiceOperator, pm partition.PartitionMetadata) { + // Cleanup verification + }, + ) +} diff --git a/pkg/tests/partition/partition_test.go b/pkg/tests/partition/partition_test.go index 4766c28a00347..80aa369937fb1 100644 --- a/pkg/tests/partition/partition_test.go +++ b/pkg/tests/partition/partition_test.go @@ -72,7 +72,7 @@ func runPartitionTableCreateAndDeleteTestsWithAware( t.Name(), cn, ) - require.Equal(t, 2, len(metadata.Partitions)) + require.NotEqual(t, 0, len(metadata.Partitions)) require.Equal(t, method, metadata.Method) var tables []string diff --git a/proto/plan.proto b/proto/plan.proto index 546198a628179..0867bd8f2a186 100644 --- a/proto/plan.proto +++ b/proto/plan.proto @@ -1426,6 +1426,7 @@ message AlterTable { string RawSQL = 12; AlterCopyOpt Options = 13; repeated string AffectedCols = 14; + AlterPartitionOption alterPartition = 15; } message DropTable { @@ -1651,4 +1652,15 @@ message CloneTable { ObjectRef src_obj_def = 4; string dst_database_name = 5; string dst_table_name = 6; +} + +message AlterPartitionOption { + AlterPartitionType AlterType = 1; +} + +enum AlterPartitionType { + AddPartitionTables = 0; + DropPartitionTables = 1; + TruncatePartitionTables = 2; + RedefinePartitionTables = 3; } \ No newline at end of file