From 66574f0dd0f05a9d2e867a06e5f7701b2ece20d8 Mon Sep 17 00:00:00 2001 From: Gao Hongtao Date: Wed, 30 Oct 2024 23:06:09 +0000 Subject: [PATCH 1/6] init Signed-off-by: Gao Hongtao --- api/proto/banyandb/database/v1/schema.proto | 2 + banyand/internal/storage/index.go | 52 +++--- banyand/internal/storage/index_test.go | 2 +- banyand/internal/storage/storage.go | 2 +- banyand/measure/query.go | 168 ++++++++++++++++-- banyand/measure/write.go | 35 ++-- docs/api-reference.md | 1 + pkg/index/index.go | 10 +- pkg/index/inverted/inverted.go | 31 ++-- pkg/index/inverted/inverted_series.go | 33 +++- pkg/partition/index.go | 25 ++- pkg/query/model/model.go | 20 +++ ..._space.inuse_objects.inuse_space.001.pb.gz | Bin 0 -> 550417 bytes 13 files changed, 298 insertions(+), 83 deletions(-) create mode 100644 pprof.banyand.alloc_objects.alloc_space.inuse_objects.inuse_space.001.pb.gz diff --git a/api/proto/banyandb/database/v1/schema.proto b/api/proto/banyandb/database/v1/schema.proto index db7f2d503..83fbeb238 100644 --- a/api/proto/banyandb/database/v1/schema.proto +++ b/api/proto/banyandb/database/v1/schema.proto @@ -115,6 +115,8 @@ message Measure { string interval = 5; // updated_at indicates when the measure is updated google.protobuf.Timestamp updated_at = 6; + // non_time_series indicates whether the measure is a time series + bool non_time_series = 7; } message MeasureAggregateFunction { diff --git a/banyand/internal/storage/index.go b/banyand/internal/storage/index.go index af6b6424c..fc5083c38 100644 --- a/banyand/internal/storage/index.go +++ b/banyand/internal/storage/index.go @@ -38,7 +38,7 @@ func (s *segment[T, O]) IndexDB() IndexDB { } func (s *segment[T, O]) Lookup(ctx context.Context, series []*pbv1.Series) (pbv1.SeriesList, error) { - sl, _, err := s.index.filter(ctx, series, nil, nil) + sl, _, _, err := s.index.filter(ctx, series, nil, nil) return sl, err } @@ -80,17 +80,17 @@ var rangeOpts = index.RangeOpts{} func (s *seriesIndex) filter(ctx context.Context, series []*pbv1.Series, projection []index.FieldKey, secondaryQuery index.Query, -) (sl pbv1.SeriesList, fields FieldResultList, err error) { +) (sl pbv1.SeriesList, fields FieldResultList, tss []int64, err error) { seriesMatchers := make([]index.SeriesMatcher, len(series)) for i := range series { seriesMatchers[i], err = convertEntityValuesToSeriesMatcher(series[i]) if err != nil { - return nil, nil, err + return nil, nil, nil, err } } indexQuery, err := s.store.BuildQuery(seriesMatchers, secondaryQuery) if err != nil { - return nil, nil, err + return nil, nil, nil, err } tracer := query.GetTracer(ctx) if tracer != nil { @@ -109,13 +109,13 @@ func (s *seriesIndex) filter(ctx context.Context, series []*pbv1.Series, } ss, err := s.store.Search(ctx, projection, indexQuery) if err != nil { - return nil, nil, err + return nil, nil, nil, err } - sl, fields, err = convertIndexSeriesToSeriesList(ss, len(projection) > 0) + sl, fields, tss, err = convertIndexSeriesToSeriesList(ss, len(projection) > 0) if err != nil { - return nil, nil, errors.WithMessagef(err, "failed to convert index series to series list, matchers: %v, matched: %d", seriesMatchers, len(ss)) + return nil, nil, nil, errors.WithMessagef(err, "failed to convert index series to series list, matchers: %v, matched: %d", seriesMatchers, len(ss)) } - return sl, fields, nil + return sl, fields, tss, nil } var emptySeriesMatcher = index.SeriesMatcher{} @@ -173,27 +173,31 @@ func convertEntityValuesToSeriesMatcher(series *pbv1.Series) (index.SeriesMatche }, nil } -func convertIndexSeriesToSeriesList(indexSeries []index.SeriesDocument, hasFields bool) (pbv1.SeriesList, FieldResultList, error) { +func convertIndexSeriesToSeriesList(indexSeries []index.SeriesDocument, hasFields bool) (pbv1.SeriesList, FieldResultList, []int64, error) { seriesList := make(pbv1.SeriesList, 0, len(indexSeries)) var fields FieldResultList if hasFields { fields = make(FieldResultList, 0, len(indexSeries)) } + var timestamps []int64 for _, s := range indexSeries { var series pbv1.Series series.ID = s.Key.ID if err := series.Unmarshal(s.Key.EntityValues); err != nil { - return nil, nil, errors.WithMessagef(err, "failed to unmarshal series: %s", s.Key.EntityValues) + return nil, nil, nil, errors.WithMessagef(err, "failed to unmarshal series: %s", s.Key.EntityValues) } seriesList = append(seriesList, &series) if fields != nil { fields = append(fields, s.Fields) } + if s.Timestamp > 0 { + timestamps = append(timestamps, s.Timestamp) + } } - return seriesList, fields, nil + return seriesList, fields, timestamps, nil } -func (s *seriesIndex) Search(ctx context.Context, series []*pbv1.Series, opts IndexSearchOpts) (sl pbv1.SeriesList, frl FieldResultList, err error) { +func (s *seriesIndex) Search(ctx context.Context, series []*pbv1.Series, opts IndexSearchOpts) (sl pbv1.SeriesList, frl FieldResultList, tss []int64, err error) { tracer := query.GetTracer(ctx) if tracer != nil { var span *query.Span @@ -207,17 +211,15 @@ func (s *seriesIndex) Search(ctx context.Context, series []*pbv1.Series, opts In } if opts.Order == nil || opts.Order.Index == nil { - var seriesList pbv1.SeriesList - var fieldResultList FieldResultList if opts.Query != nil { - seriesList, fieldResultList, err = s.filter(ctx, series, opts.Projection, opts.Query) + sl, frl, tss, err = s.filter(ctx, series, opts.Projection, opts.Query) } else { - seriesList, fieldResultList, err = s.filter(ctx, series, opts.Projection, nil) + sl, frl, tss, err = s.filter(ctx, series, opts.Projection, nil) } if err != nil { - return nil, nil, err + return nil, nil, nil, err } - return seriesList, fieldResultList, nil + return sl, frl, tss, nil } fieldKey := index.FieldKey{ @@ -238,17 +240,17 @@ func (s *seriesIndex) Search(ctx context.Context, series []*pbv1.Series, opts In for i := range series { seriesMatchers[i], err = convertEntityValuesToSeriesMatcher(series[i]) if err != nil { - return nil, nil, err + return nil, nil, nil, err } } query, err := s.store.BuildQuery(seriesMatchers, opts.Query) if err != nil { - return nil, nil, err + return nil, nil, nil, err } iter, err := s.store.Iterator(ctx, fieldKey, rangeOpts, opts.Order.Sort, opts.PreloadSize, query, opts.Projection) if err != nil { - return nil, nil, err + return nil, nil, nil, err } defer func() { err = multierr.Append(err, iter.Close()) @@ -265,16 +267,16 @@ func (s *seriesIndex) Search(ctx context.Context, series []*pbv1.Series, opts In doc.Key.EntityValues = val.EntityValues result = append(result, doc) } - sortedSeriesList, sortedFieldResultList, err := convertIndexSeriesToSeriesList(result, len(opts.Projection) > 0) + sl, frl, tss, err = convertIndexSeriesToSeriesList(result, len(opts.Projection) > 0) if err != nil { - return nil, nil, errors.WithMessagef(err, "failed to convert index series to series list, matchers: %v, matched: %d", seriesMatchers, len(result)) + return nil, nil, nil, errors.WithMessagef(err, "failed to convert index series to series list, matchers: %v, matched: %d", seriesMatchers, len(result)) } if span != nil { span.Tagf("query", "%s", iter.Query().String()) span.Tagf("rounds", "%d", r) - span.Tagf("size", "%d", len(sortedSeriesList)) + span.Tagf("size", "%d", len(sl)) } - return sortedSeriesList, sortedFieldResultList, err + return sl, frl, tss, err } func (s *seriesIndex) Close() error { diff --git a/banyand/internal/storage/index_test.go b/banyand/internal/storage/index_test.go index 304bcff9d..6e4db479a 100644 --- a/banyand/internal/storage/index_test.go +++ b/banyand/internal/storage/index_test.go @@ -157,7 +157,7 @@ func TestSeriesIndex_Primary(t *testing.T) { seriesQuery.EntityValues = tt.entityValues[i] seriesQueries = append(seriesQueries, &seriesQuery) } - sl, _, err := si.filter(ctx, seriesQueries, nil, nil) + sl, _, _, err := si.filter(ctx, seriesQueries, nil, nil) require.NoError(t, err) require.Equal(t, len(tt.entityValues), len(sl)) assert.Equal(t, tt.subject, sl[0].Subject) diff --git a/banyand/internal/storage/storage.go b/banyand/internal/storage/storage.go index 4bcc43d8f..42502515c 100644 --- a/banyand/internal/storage/storage.go +++ b/banyand/internal/storage/storage.go @@ -81,7 +81,7 @@ type FieldResultList []FieldResult // IndexDB is the interface of index database. type IndexDB interface { Write(docs index.Documents) error - Search(ctx context.Context, series []*pbv1.Series, opts IndexSearchOpts) (pbv1.SeriesList, FieldResultList, error) + Search(ctx context.Context, series []*pbv1.Series, opts IndexSearchOpts) (pbv1.SeriesList, FieldResultList, []int64, error) } // TSDB allows listing and getting shard details. diff --git a/banyand/measure/query.go b/banyand/measure/query.go index 299402fe1..457e90660 100644 --- a/banyand/measure/query.go +++ b/banyand/measure/query.go @@ -86,29 +86,36 @@ func (s *measure) Query(ctx context.Context, mqo model.MeasureQueryOptions) (mqr EntityValues: mqo.Entities[i], } } - var result queryResult - result.ctx = ctx + tsdb := db.(storage.TSDB[*tsTable, option]) - result.segments = tsdb.SelectSegments(*mqo.TimeRange) - if len(result.segments) < 1 { - return &result, nil + segments := tsdb.SelectSegments(*mqo.TimeRange) + if len(segments) < 1 { + return model.BypassResult, nil + } + + if s.schema.NonTimeSeries { + return s.buildIndexQueryResult(ctx, series, mqo, segments) } - defer func() { - if err != nil { - result.Release() - } - }() - sids, tables, storedIndexValue, newTagProjection, err := s.searchSeriesList(ctx, series, mqo, result.segments) + sids, tables, storedIndexValue, newTagProjection, err := s.searchSeriesList(ctx, series, mqo, segments) if err != nil { return nil, err } if len(sids) < 1 { - return &result, nil + return model.BypassResult, nil + } + result := queryResult{ + ctx: ctx, + segments: segments, + tagProjection: newTagProjection, + storedIndexValue: storedIndexValue, } - result.tagProjection = mqo.TagProjection + defer func() { + if err != nil { + result.Release() + } + }() mqo.TagProjection = newTagProjection - result.storedIndexValue = storedIndexValue var parts []*part qo := queryOptions{ MeasureQueryOptions: mqo, @@ -195,7 +202,7 @@ func (s *measure) searchSeriesList(ctx context.Context, series []*pbv1.Series, m } seriesFilter := roaring.NewPostingList() for i := range segments { - sll, fieldResultList, err := segments[i].IndexDB().Search(ctx, series, storage.IndexSearchOpts{ + sll, fieldResultList, _, err := segments[i].IndexDB().Search(ctx, series, storage.IndexSearchOpts{ Query: mqo.Query, Order: mqo.Order, PreloadSize: preloadSize, @@ -239,6 +246,46 @@ func (s *measure) searchSeriesList(ctx context.Context, series []*pbv1.Series, m return sl, tables, storedIndexValue, newTagProjection, nil } +func (s *measure) buildIndexQueryResult(ctx context.Context, series []*pbv1.Series, mqo model.MeasureQueryOptions, + segments []storage.Segment[*tsTable, option], +) (*indexQueryResult, error) { + r := &indexQueryResult{ + ctx: ctx, + series: series, + mqo: mqo, + segments: segments, + } + for _, tp := range mqo.TagProjection { + tagFamilyLocation := tagFamilyLocation{ + name: tp.Family, + fieldToValueType: make(map[string]tagNameWithType), + projectedEntityOffsets: make(map[string]int), + } + TAG: + for _, n := range tp.Names { + for i := range s.schema.GetEntity().GetTagNames() { + if n == s.schema.GetEntity().GetTagNames()[i] { + tagFamilyLocation.projectedEntityOffsets[n] = i + continue TAG + } + } + if fields, ok := s.fieldIndexLocation[tp.Family]; ok { + if field, ok := fields[n]; ok { + r.indexProjection = append(r.indexProjection, field.Key) + tagFamilyLocation.fieldToValueType[field.Key.Marshal()] = tagNameWithType{ + name: n, + typ: field.Type, + } + continue TAG + } + } + return nil, fmt.Errorf("tag %s not found in schema", n) + } + r.tfl = append(r.tfl, tagFamilyLocation) + } + return r, nil +} + func (s *measure) searchBlocks(ctx context.Context, result *queryResult, sids []common.SeriesID, parts []*part, qo queryOptions) error { bma := generateBlockMetadataArray() defer releaseBlockMetadataArray(bma) @@ -634,3 +681,94 @@ func (qr *queryResult) merge(storedIndexValue map[common.SeriesID]map[string]*mo return result } + +var ( + bypassVersions = []int64{0} + bypassFields = []model.Field{} +) + +type indexQueryResult struct { + mqo model.MeasureQueryOptions + tfl []tagFamilyLocation + indexProjection []index.FieldKey + ctx context.Context + series []*pbv1.Series + + segments []storage.Segment[*tsTable, option] + + i int + sll pbv1.SeriesList + frl storage.FieldResultList + timestamps []int64 + err error +} + +// Pull implements model.MeasureQueryResult. +func (i *indexQueryResult) Pull() *model.MeasureResult { + if i.i < 0 { + if len(i.segments) < 1 { + return nil + } + i.sll, i.frl, i.timestamps, i.err = i.segments[0].IndexDB().Search(i.ctx, i.series, storage.IndexSearchOpts{ + Query: i.mqo.Query, + Order: i.mqo.Order, + PreloadSize: preloadSize, + Projection: i.indexProjection, + }) + if i.err != nil { + return &model.MeasureResult{ + Error: i.err, + } + } + i.segments = i.segments[1:] + if len(i.sll) < 1 { + return i.Pull() + } + i.i = 0 + } + if i.i >= len(i.sll) { + i.i = -1 + return i.Pull() + } + r := &model.MeasureResult{ + SID: i.sll[i.i].ID, + Timestamps: []int64{i.timestamps[i.i]}, + Versions: bypassVersions, + } + for j := range i.tfl { + tagFamily := model.TagFamily{Name: i.tfl[j].name} + for name, offset := range i.tfl[j].projectedEntityOffsets { + tagFamily.Tags = append(tagFamily.Tags, model.Tag{ + Name: name, + Values: []*modelv1.TagValue{i.sll[i.i].EntityValues[offset]}, + }) + } + if i.frl == nil { + continue + } + for f, v := range i.frl[j] { + if tnt, ok := i.tfl[j].fieldToValueType[f]; ok { + tagFamily.Tags = append(tagFamily.Tags, model.Tag{ + Name: tnt.name, + Values: []*modelv1.TagValue{mustDecodeTagValue(tnt.typ, v)}, + }) + } else { + return &model.MeasureResult{ + Error: errors.Errorf("unknown field %s not found in fieldToValueType", f), + } + } + } + r.TagFamilies = append(r.TagFamilies, tagFamily) + } + i.i++ + return r +} + +func (i *indexQueryResult) Release() { +} + +type tagFamilyLocation struct { + name string + fieldToValueType map[string]tagNameWithType + projectedEntityOffsets map[string]int +} diff --git a/banyand/measure/write.go b/banyand/measure/write.go index 2eaaffddd..13d5aec02 100644 --- a/banyand/measure/write.go +++ b/banyand/measure/write.go @@ -156,25 +156,33 @@ func (w *writeCallback) handle(dst map[string]*dataPointsInGroup, writeEvent *me t.Name, t.Type, tagValue) - if r, ok := tfr[t.Name]; ok { + r, ok := tfr[t.Name] + if ok || stm.schema.NonTimeSeries { + fieldKey := index.FieldKey{} + switch { + case ok: + fieldKey.IndexRuleID = r.GetMetadata().GetId() + fieldKey.Analyzer = r.Analyzer + case stm.schema.NonTimeSeries: + fieldKey.TagName = t.Name + default: + logger.Panicf("metadata crashed, tag family rule %s not found", t.Name) + } + toIndex := ok || !stm.schema.NonTimeSeries if encodeTagValue.value != nil { fields = append(fields, index.Field{ - Key: index.FieldKey{ - IndexRuleID: r.GetMetadata().GetId(), - Analyzer: r.Analyzer, - }, + Key: fieldKey, Term: encodeTagValue.value, Store: true, + Index: toIndex, }) } else { for _, val := range encodeTagValue.valueArr { fields = append(fields, index.Field{ - Key: index.FieldKey{ - IndexRuleID: r.GetMetadata().GetId(), - Analyzer: r.Analyzer, - }, + Key: fieldKey, Term: val, Store: true, + Index: toIndex, }) } } @@ -203,11 +211,16 @@ func (w *writeCallback) handle(dst map[string]*dataPointsInGroup, writeEvent *me }) } - dpg.docs = append(dpg.docs, index.Document{ + doc := index.Document{ DocID: uint64(series.ID), EntityValues: series.Buffer, Fields: fields, - }) + } + if stm.schema.NonTimeSeries { + doc.Timestamp = ts + } + dpg.docs = append(dpg.docs, doc) + return dst, nil } diff --git a/docs/api-reference.md b/docs/api-reference.md index 852c7f667..eb008338c 100644 --- a/docs/api-reference.md +++ b/docs/api-reference.md @@ -1076,6 +1076,7 @@ Measure intends to store data point | entity | [Entity](#banyandb-database-v1-Entity) | | entity indicates which tags will be to generate a series and shard a measure | | interval | [string](#string) | | interval indicates how frequently to send a data point valid time units are "ns", "us" (or "µs"), "ms", "s", "m", "h", "d". | | updated_at | [google.protobuf.Timestamp](#google-protobuf-Timestamp) | | updated_at indicates when the measure is updated | +| non_time_series | [bool](#bool) | | non_time_series indicates whether the measure is a time series | diff --git a/pkg/index/index.go b/pkg/index/index.go index 644909916..0b2255810 100644 --- a/pkg/index/index.go +++ b/pkg/index/index.go @@ -51,10 +51,14 @@ type FieldKey struct { Analyzer string SeriesID common.SeriesID IndexRuleID uint32 + TagName string } // Marshal encodes f to string. func (f FieldKey) Marshal() string { + if len(f.TagName) > 0 { + return f.TagName + } return string(convert.Uint32ToBytes(f.IndexRuleID)) } @@ -69,6 +73,7 @@ type Field struct { Key FieldKey NoSort bool Store bool + Index bool } // RangeOpts contains options to performance a continuous scan. @@ -214,8 +219,9 @@ func (s Series) String() string { // SeriesDocument represents a series document in an index. type SeriesDocument struct { - Fields map[string][]byte - Key Series + Fields map[string][]byte + Timestamp int64 + Key Series } // SeriesStore is an abstract of a series repository. diff --git a/pkg/index/inverted/inverted.go b/pkg/index/inverted/inverted.go index 5d35d6aa6..167486781 100644 --- a/pkg/index/inverted/inverted.go +++ b/pkg/index/inverted/inverted.go @@ -58,7 +58,6 @@ var ( defaultUpper = convert.Uint64ToBytes(math.MaxUint64) defaultLower = convert.Uint64ToBytes(0) defaultRangePreloadSize = 1000 - defaultProjection = []string{docIDField} ) // Analyzers is a map that associates each IndexRule_Analyzer type with a corresponding Analyzer. @@ -272,7 +271,7 @@ func (s *store) MatchTerms(field index.Field) (list posting.List, err error) { if err != nil { return nil, err } - iter := newBlugeMatchIterator(documentMatchIterator, reader, defaultProjection) + iter := newBlugeMatchIterator(documentMatchIterator, reader, nil) defer func() { err = multierr.Append(err, iter.Close()) }() @@ -305,7 +304,7 @@ func (s *store) Match(fieldKey index.FieldKey, matches []string, opts *modelv1.C if err != nil { return nil, err } - iter := newBlugeMatchIterator(documentMatchIterator, reader, defaultProjection) + iter := newBlugeMatchIterator(documentMatchIterator, reader, nil) defer func() { err = multierr.Append(err, iter.Close()) }() @@ -352,16 +351,19 @@ type blugeMatchIterator struct { needToLoadFields []string current index.DocumentResult hit int + ctx *search.Context } func newBlugeMatchIterator(delegated search.DocumentMatchIterator, closer io.Closer, needToLoadFields []string, ) blugeMatchIterator { + needToLoadFields = append(needToLoadFields, entityField, docIDField, seriesIDField, timestampField) bmi := blugeMatchIterator{ delegated: delegated, closer: closer, needToLoadFields: needToLoadFields, current: index.DocumentResult{Values: make(map[string][]byte, len(needToLoadFields))}, + ctx: search.NewSearchContext(1, 0), } for _, f := range needToLoadFields { bmi.current.Values[f] = nil @@ -391,8 +393,19 @@ func (bmi *blugeMatchIterator) Next() bool { if len(match.SortValue) > 0 { bmi.current.SortedValue = match.SortValue[0] } - err := match.VisitStoredFields(func(field string, value []byte) bool { - switch field { + err := match.LoadDocumentValues(bmi.ctx, bmi.needToLoadFields) + + if err != nil { + bmi.err = errors.WithMessagef(err, "load document values, hit: %d", bmi.hit) + return false + } + for i := range bmi.needToLoadFields { + vv := match.DocValues(bmi.needToLoadFields[i]) + if vv == nil { + continue + } + value := vv[0] + switch bmi.needToLoadFields[i] { case entityField: bmi.current.EntityValues = value case docIDField: @@ -407,13 +420,11 @@ func (bmi *blugeMatchIterator) Next() bool { } bmi.current.Timestamp = ts.UnixNano() default: - if _, ok := bmi.current.Values[field]; ok { - bmi.current.Values[field] = bytes.Clone(value) + if _, ok := bmi.current.Values[bmi.needToLoadFields[i]]; ok { + bmi.current.Values[bmi.needToLoadFields[i]] = bytes.Clone(value) } } - return true - }) - bmi.err = errors.WithMessagef(err, "visit stored fields, hit: %d", bmi.hit) + } return bmi.err == nil } diff --git a/pkg/index/inverted/inverted_series.go b/pkg/index/inverted/inverted_series.go index 7288dd315..dd02dd02b 100644 --- a/pkg/index/inverted/inverted_series.go +++ b/pkg/index/inverted/inverted_series.go @@ -21,6 +21,7 @@ package inverted import ( "bytes" "context" + "fmt" "github.com/blugelabs/bluge" "github.com/blugelabs/bluge/search" @@ -117,11 +118,13 @@ func parseResult(dmi search.DocumentMatchIterator, loadedFields []index.FieldKey return nil, errors.WithMessage(err, "iterate document match iterator") } docIDMap := make(map[uint64]struct{}) - fields := make([]string, 0, len(loadedFields)) + fields := make([]string, 0, len(loadedFields)+3) + fields = append(fields, docIDField, entityField, timestampField) for i := range loadedFields { fields = append(fields, loadedFields[i].Marshal()) } var hitNumber int + ctx := search.NewSearchContext(1, 0) for err == nil && next != nil { hitNumber = next.HitNumber var doc index.SeriesDocument @@ -131,8 +134,17 @@ func parseResult(dmi search.DocumentMatchIterator, loadedFields []index.FieldKey doc.Fields[fields[i]] = nil } } - err = next.VisitStoredFields(func(field string, value []byte) bool { - switch field { + err = next.LoadDocumentValues(ctx, fields) + if err != nil { + return nil, errors.WithMessagef(err, "visit stored fields, hit: %d", hitNumber) + } + for i := range fields { + vv := next.DocValues(fields[i]) + if vv == nil { + continue + } + value := vv[0] + switch fields[i] { case docIDField: id := convert.BytesToUint64(value) if _, ok := docIDMap[id]; !ok { @@ -140,16 +152,19 @@ func parseResult(dmi search.DocumentMatchIterator, loadedFields []index.FieldKey docIDMap[id] = struct{}{} } case entityField: + fmt.Printf("entityField: %v\n", vv) doc.Key.EntityValues = value + case timestampField: + ts, errTime := bluge.DecodeDateTime(value) + if errTime != nil { + return nil, err + } + doc.Timestamp = ts.UnixNano() default: - if _, ok := doc.Fields[field]; ok { - doc.Fields[field] = bytes.Clone(value) + if _, ok := doc.Fields[fields[i]]; ok { + doc.Fields[fields[i]] = bytes.Clone(value) } } - return true - }) - if err != nil { - return nil, errors.WithMessagef(err, "visit stored fields, hit: %d", hitNumber) } if doc.Key.ID > 0 { result = append(result, doc) diff --git a/pkg/partition/index.go b/pkg/partition/index.go index 9ccb08691..133dc6362 100644 --- a/pkg/partition/index.go +++ b/pkg/partition/index.go @@ -64,18 +64,25 @@ func ParseIndexRuleLocators(entity *databasev1.Entity, families []*databasev1.Ta ir := findIndexRuleByTagName(families[i].Tags[j].Name) if ir != nil { ttr[families[i].Tags[j].Name] = ir - tagFamily, ok := fil[families[i].Name] - if !ok { - tagFamily = make(map[string]FieldWithType) - fil[families[i].Name] = tagFamily + } + tagFamily, ok := fil[families[i].Name] + if !ok { + tagFamily = make(map[string]FieldWithType) + fil[families[i].Name] = tagFamily + } + fwt := FieldWithType{ + Type: pbv1.MustTagValueSpecToValueType(families[i].Tags[j].Type), + } + if ir != nil { + fwt.Key = index.FieldKey{ + IndexRuleID: ir.Metadata.Id, } - tagFamily[families[i].Tags[j].Name] = FieldWithType{ - Key: index.FieldKey{ - IndexRuleID: ir.Metadata.Id, - }, - Type: pbv1.MustTagValueSpecToValueType(families[i].Tags[j].Type), + } else { + fwt.Key = index.FieldKey{ + TagName: families[i].Tags[j].Name, } } + tagFamily[families[i].Tags[j].Name] = fwt } } return locators, fil diff --git a/pkg/query/model/model.go b/pkg/query/model/model.go index 022bcdbc4..c307918d1 100644 --- a/pkg/query/model/model.go +++ b/pkg/query/model/model.go @@ -98,6 +98,26 @@ type MeasureQueryResult interface { Release() } +var ( + BypassResult = &bypassResult{} + dummyResult = &MeasureResult{} +) + +// bypassResult struct. +type bypassResult struct { + // Add fields as necessary +} + +// Implement Pull method. +func (b *bypassResult) Pull() *MeasureResult { + return dummyResult +} + +// Implement Release method. +func (b *bypassResult) Release() { + // No operation +} + // StreamQueryOptions is the options of a stream query. type StreamQueryOptions struct { Name string diff --git a/pprof.banyand.alloc_objects.alloc_space.inuse_objects.inuse_space.001.pb.gz b/pprof.banyand.alloc_objects.alloc_space.inuse_objects.inuse_space.001.pb.gz new file mode 100644 index 0000000000000000000000000000000000000000..2dfb0d5ce520972e76b2e40336dd94c35c524ff3 GIT binary patch literal 550417 zcmV)#K##v4iwFP!00000|IEF2d{oD|H=a3tpItyKLI@B-T}bp|ic2ELC3)i%CpX2; z{UyP6;#@n)bxFDRjdOF4=%NTwM8}8@qBqlQSbA@!m}+EtHNBbk`|PfE=FD?u&aQ}^ zH-Fg9Cz^A1&XlKqpYH>jMUoWIEHz03&C(q-uwHWglm@8{8#PXAlHN2UGb=kMXe3n@XMXK#jSNdmwQVKX92(~4>E)mQ`~UZgU;Zj+o*;G!{aY}Y z1NFMT^6G29{ta5R-*+EEN96?@gYWg<{>K|{{_gjG_)ph#8iVhxx7i>01_y@Uk5nQbmFdK(|R`ZhIBG7kxGmYzE22q-8hKo%A5w z_qCa@{m$cG`&uJ2vZZ54untYZ{b|J4_$+u zA-{z5CH+W$^1=WTTVu&Ox1UCzd@-00^-IY>;y!`qUI-P_?^45SQbq>xpJZo*=$hCh z_#`wROoou5q?`!P(lEl!1Z`IvX!_2(!^sFTl8hpw$rv)0xDNV^pNyb$Zxd_>8Sj&E zq=Jkm6Uam|iA*L_$W$_oOed8@GRIs5Sx45B z4P;|%tns}Kqv#Jj7R-fu;16sfo5>dL2&}a((+#RpgBehJE7?Z2lO1Fyi5}|%_0G_5 z-${Z`Lz`XXW3rpT75IO9zew%WZ&}qLN%!H&P+wLSeMNZQX zLO1Js@Weaso*`$+IdYy{AQ#E!7$e=eU!dR_{8*6JkE0(`1>ox$^a!QQH}5R9!-yM z$FTYWf1p$usFW##l)=gnWvEiF3{!?HBNUftE`-#P$|z;DGDaDzyswN?DwOfc1ZAQ! zNeLSDk5*;GRwG2hQfaa>MTvcLRG5=)W%P9!!FG@tz423(Y07k^QkkL5RAwo&l{v~> zWnOH){PlkfvgRubl!eM7#p@Jz-EsZ6O7}*umgp~jpe$CFC{b~`REf#cHDpy^`V;ko z9Uya=vRwI4`ACtUF7xdkvt|b9FVqfZL){fh=aouGap)$z#T_AQRs7jo%gFVL$);oa z9U*IV0?#cVoA-BfKGz1CtWi9#`=5rb+iTHn^Tq6jwF=eYY@N0>+)1x;1(EC1UtC`E z`8zRv8X6r|KZWZQPUoEBA=&hvGl7 zF5V+(V(k;U??|UV4$amnVI6WtIjcNf{cqVypo`NGtaJ3B@|q?c0_P(_@Pcwt3EQxf z6}vY*$F+lxLC)vOCFQbm#UU`SD%X_j$_-8$zuKdmK33?db&-s33bk)4w-l!}$iFMY zHNc!fiu=Q~V<4w3Mt-cekx7JkT9>^^kjNx`o}3*o$_ zCkg%Ncme~7)7<|mmLUGpc)Ash0lrY~D&jwWAD+CY$gI%&$^+%0Vyne!H&tqrY9S`N z&0sDByQ?i-%N|&Kf2~55PQ&0Z0zLkz>n1-D{Cea<@IGU3HtM?@LRr)c!yZ<;e8>S9dN2nv!QR--Q zjLIGytG>$HeQ{ePE;o)@azI3#9~i+uqo7fi8nH@ zDiHWhEFAVTG3$f5%) zlha$ygqS<@=jwRPe071kP<6&@ko}T~*@@zkiI|SEO$6Xv%g7wRohm#WLu z>W}&33}V^rxNk( z(Nl?tG4w?pAj|bgLeop+sZz6oMTE7m#)^f(;yk!nD=sZ!ZgU}|-c}^G54M;(Le3fm z2`k}m%vHLNE@EcUW02#t+Y^l~KMU<}SULe~3ZMFP2<%b!s;6Z^;AW5#7J&QIPt^VD z0Z#ZKTfQmOjzu62s)y8H{wiqXk6lE!?#*CZNM|uc@e%pk?efnxM;Nj*6rEGeim628Bph#ijr#l#x>@QWFxm?34Q$S zV25fcN)kidO^LYx{gXdq^}2dP zy{U?lX!u8t&7nUMs^f~^;YmCJg}2n(>K*k9^&q*c#!ZoOx-uK;p0Cc3+*9XCk}A27 zd|tV)K2RU3wpOe$i>Rn?1szrt{5rJgrghhPXg#%FT5qk77V{}fw7#0qujI`#aV+Df zU(C4j9l9L<8J4^Kw1uRY9r|2rS2^L5)57r-V zteM@P&{H!ylFC2t2$Gg*&LAl{w&@piwm%CC$w3-REf01qUPfhzCb#TJa%+h88_t5p zwstE>Tc>dH2Kjdvd^?%Z8S(F8OBKtnAv4uT-LjWXqLKvbTSGPLS1wm118SFRT%yqk zcDq6!tRlhY&~TVGTpOW{)F_f-lr~x$qm9)BS)P>lwdm%J(<(G)wjIqU2a>=)UgLcM zQoMOE+mJzF{Zea!Hc?|)L8DgR3Dl$yDPdcEk~UeJqD|GNY16ezZHDIXF0tavfZ8*) zSsI_b;cj*&)QL^&r}R9{IMj*$WIjG5d?a1HvpbKD4Fo$w-l^)q|7^|MKtOrLEJ&K8 z&DD6nYnOj6^YWFj7>c^VbJTg-d~Kd%6M6mgO=!#e%Zbbgy(5ZrTA(e|7HJ=7v(&}f z5{(1a@E0@#YKMKorP?wrc7>6rC(Q`HRI#_@cQhk7Dl&4pCc449lnd);rPKBH2lF8L zL(Nu-6?FRCq2_wSw2w3wSO%pxGNASfZKW2{R%tPa5qeQN1j3)WT3e%43t0XmdGHd# zKNUb~*v?w3t#jWOB+WA+X}z{V+o)~QHq*<;4=I2KU0>OvZPm7Eu4{^{)*J|K*LG+- zwOvjLhoyGH2DH)TeA&~nT_}ryb3WFVs=Kv4+Fs46rg?KM3zGI}pJ*rq$IGWCSVeX! zolO7y9if^(`tf7*MC61|#D497c2GN{9oCL$pK1~tq#2|fjm)`9`wY!`gBoIo!skej zxdV5N?`)<*t~`>>Jr>as9)cR5Ik8||xq1<`Z$gWdU>?*5f8e-wLc`!WK6aP|Nhh^a z+G)-4i&*A$g4|V)w9ic-a7J^6#(3jLp!e|h7DB^PGLW3r&M|4iPAw0P%Y>x!_{bt} zBL{*Pw2Rv3+9mC>c15ekgjvP5)yFQXevxd2ZM&yVRZq`|!~V{@SG8-}b?t_BQ@bM! z`l4810W`R!-PZ1CUubu=dm0xELt&#dfcx45?V)Dt#kwnMUI-0ilD(V0SBrvNy6Zxa zOCBWm&_yR4Nt*<{=#TxOo<$5-6^lW51)mWVD+6lx)Gp{2D8yaMeAOk-rVwj^J z!Z&uPF<4hGy|>;+FVXwz{q+9&0M~zSky1T=)F&MR1NAa}kUm%+qK7pTvSQjo#!&q! z>{FlcO1Bh3s?!?efQI;#NEP2}hi>-$Z+Zqtk?_`*>%;WnIx;ix2MLV@bl{>f9vU!^ zeHIG6L=3vvK8?_wR}VQe%^)T0>Ig08n`d67BY)vJ^3J*+(UISY0kKBvqjY!X010rR zzk>yG3h1NtF}f>qP!Rc{O~euFr(y_s(NYt`?bMFd-`B_K75aF6f?mBlRN!DC{0?;e zvU1vq=BxV>`0rz1aUNYd7ySK_q)fw!_hvVsC*n`RF$7A;M17JzS+64h50bU`E07;I zmB3KqmGAt2H&Y}sa-nW_6}#ltAToCG?8el0O_g}==HgAG4La&SgMWoLrs&`LlJn;Y zX2>@XSVX4k)AT+{i85V}B3=JA|5smS52@5==reVp4#|R~SvoR4@d0b<0xu#HD8-ts zTVbmeX@3Fg5!to*5tDL`K3B)-KIC?0L*04$d>vVKNQX+JF7J-qd!d9n`Js^!Ew?~l zs0%D(KFuhK``MvU8gmsjkwak4DS3fP5TG!8oYqA+zYM6oNdG`ztS`}*>dW-yx(kNi z3eu|MuOI5d=qm~XwS>k$a{((H$9i93sJMvvXq@K#h||32#o17Ig^slr{+?_IuGHOL zCa(tbAlX@v!*nf5t>i&+NN4Ud8+R&D!G2T;>+i6GyGppb==ePk9faZcxOf9}L&Vno zoSvgte0Y=+_DxpnyjhJS+0CKh8eO(qRQJ*47@~gcQ2!h}iLI0gblFSHdc`l?r;OGJ+}A_NXjJUUM@@*`e>$ zcj+JN=v#4Jg(yCfW1o?|rw?=A`-taU9J$vI_gF@O|JgXaQM{Z~=3Rk7^BYByh0kwDY-a7uVW{|Q+eB5v}v^p!ZS7%5smX#S- z+KMWzhQY=FZ@s-wk3~j#n@;?BJK3V|lr8#BgD(0`Lo((h+nAH$jj==5*iv4TjIl?u z12-G!69N!u7;Fd0Jj>j(6MI*xwDsi?-d!2)OTMw}v! zXEuF!fnlxHU=ITC9@4)mzb;`P zh7b39;w6+#V`yz|sFqjI19>)0q1I`guNUW?r0P%q;g70B)~G&xXEyzGGnfr^Vz`wv zv2Xwxc+PKDQB;6Vntw((s@Rljh^xh21(6MrfEQ(;jsTe+vw$OHMXw1WNAxUl~z z7&KsMF}-W{|3=cQXpBMD+npEb)Dc$)Dip|upd(yH@j~s(rqMw!2AhEYbN!O;;Bnxrxf}@I(k|;)^sD+coyA*F zG&Ki;*J%ogh1*cXsy$?IU}WK!{$S3v<9bI+r z5?Q?LksLbaf{F9YbI|^VhZ>ZMeMIatg}@MUQ?E$@<3CA1xv@C6bVhv<152TtaVN;T zttVCo$h-GXL+e<{L^OGRNB=^{j?W3tbh!*KF_W2GGqvC|k5=f%&o&(bclCRYn0s+i zMYpbOu?ivezWzXesN04MIDik5r`~C_dMDK;6_NVFQwbO7qv&Qf)V-z0s*0#Kaz#cY zy7kZVF5l_PWdDdrI;7a>X1K7HSaP(2CX-dZRMT^99Zq`9U_PxOy}J>Uo{}@o{nEFg zV^~_q6M_i~kUzt(z3LslJe!#RfbYQ*-XGw7#qn223fo)Hx;oqE2#MuGN)Mx_(aUfR zHC|jlk{+h^GLt>nYt7Ci`o2E~--cg)Ig95(iNM?1e}gx_{N#7Sfe}frx_nqU%T@m& z_#V9BrC+{eVUM&@b75jxsGV_Yv>cKMk$Rj6nupz%$qwVhlCP zjbX-cV}vo%7-blv4P%U9j5REu>+rXM^!JUt{DQ)9Ms$|$m^y-IJOu@A*tT>@-#1qd z80=h9M~!U^ZCPQAHwNev3`V3ELt7?NWBU?(3rmGQQUIqnkQ1rtLP(9iIXu~k#w5do z=(EOIV+4fEgXGCZ0z$jGc5;g0hJhk*L^G+n#~iA8kbG2#r$>ppHqbPd3t{I4*>=1K zn+~<68qO#R!q>NjjA=%6jhyQ=-QXbm7ztZxu!G7EMfc-zD3FwTiXO(*1ELV`Y!=?0 zEkpy6bDy`MIW;Nvz#Q zSJvEQ$k2C#;|Y`!%y;q6cJ<@9Gua{w)gVJBy-h#l_ragSb2ZBYB$%5k@bZUwF5IV} zz+Dj{RUw0|;d6%i94bUuMEB7HT9#wyhHadM#v%h#KjY(3=o!T2flj$l?*n78%WT5x z=mqHF)NfG(HDQ`q^s+?Aj``D^;i{O&HMtp3XNj@YSY|9YJ~XPol&4{B1}R-%`N%*; znNe~Y-au^)>Cx2r3S*@aGMw;uaiw4pv|43EC9;z{szh6DNF*ZOG8cRcc^p!!^3@)b zcK?Y6LWYxIDi~{wh*rY)sK!#eSnWNxD_i0m2!=NSgKSXci5oJDYh$Ih(pn>CGu7-o zWkJ$9W4*zb<+*7$I(6>N8_XkU z!J8UCgXe1&jYzbN;TzCuyCG4C31vY44!{2@${-Sk;{O-?=1Z+;NLU>S4!Hl}YQ~tP zNDd!G_k{5#@Vi30xWoyIOhWanWlSklJ^-awQd5!C9rS?f^9HxH6`8&T*H zKhYU`l@Sz#8OdSnN#NtS>(c?UuxTgyP=tVP1L^UphPmHq2N`>7y728FV_)27_EnO( zbPd{suh=JsNY=`o0)7*lx-qwWac5h|*l!##4jPAy!^RQgQ{$-N`g)|Qo~VhDf(i0k zK%=lrRb_l;95ap^CybNEDdRLXY+Zoc2Drro4KDGrBoC5fHu@RP9Yxi6%^>Bhan3kz zTrf^)JT4x27SN#(oTC)NbnPF15Ga*SB$I1HRHN*!?-C3MTm+6 zwrIDE+sM;K_DdGjz3wV^Lk4CmNIRwTWzl>{481cS>fbTGFc`%{x`arh8KkVICTJj*+WJB&{{?&o1l;JS9n9Li@%cK zI{(|P;#~k_8@Gdu?qG&Zsff+w5b8-+@#~TWDHh_BH#N{Y}~J#y2M)>Yq{vm;#O$)EOrl^e#1p zL2oiS?9Lv94eEhr&9Vnbv7l1(5u8XsUzP<)Wo8_YRMcPK z*6rrhf3GhIOig6i!-+Cn;TV=sA0x&v_6n_J*vyq;0C$N!kYQIhNI>4l%NU+?qqt6F z#jdvWtI|bf0tj54Vhu7afg9BdnhZAc@`soOh2xB9RM*p^NQb~s(;3W1Fe0z=@k!&q zK}WU|l3xhw{K`$9pngB;J*9}*8J!_-m?`e6h3#u(I(qm2fOhb37GIonmeVFJh^N7SWfDtAOP@uTHz$-$4 zh(usbg8YMon=r9JX=`XQ()0|&xtDd6Ioga8gN!jx%SzH-fXCzez6don9c7%Wz!*GIB~Li-C63%3p><4tGF6{jfTeb`J$8loaw4}pCI z;;2#i+HX2?=y{Ml!4y(l@lI4*Xd1zjOf>5T>z&ok35mjRt*m=~ZX$aZMNCXGyE>+{ zysh_F-5z^_2)62Q>m+ls>6M&LJlrY{Zw^8D^Z3ZJ_B)(>tq%q;L_m^3*i~<$)}hmXJmP9YTm8pCX<~)=0D0qy{0acL8Wtk${565v0x)+LYHSgf(fBqojj)PUVSN7l9wQkR%AhJ#|u z?nC#?3U*AShnQ~_l+V08fH@^oukmM(f1s!2@L6x zZYAa>Q{Jzbvv{ISU0&R5j>0TYIJeyf(lPTp-frZ0r$Zn*&sZw}^XL>dQ!PBx^NN!l zlh4o9``|qN{k=q%qd!ouf_9gzP`$`lar~eKG+LrARkxU1&28p(6PpSs!uk|6-(l`F z<^DQ?{3Gg?SRamfmIiMi=R)#oU6i}v?cg?$zRUdB+-knrY4CRO+wjYpwfd|152A$qd+_F0nJ4l-w>E*dzAFBV1mmJjp>|K5BcewN?l170 zn)Y>k#EA|0KbPrIcsp23;O(!h7h-~pj!~w0XB`XAH1ER8xnvvaH87Dt8TqQ2PHP@< zN7l-7(7}sD^B?R&TL0weVfCmoen%4c^N;JhiFq}@3%&<$d^wUKp<{rAL?$HdHBn5< zRicuz&vf=tm%=`qK;RQ|zqwb7nNd6#8`)T7NN+|9(c0ap1^qn*ben{YTm(tY7TPez zE0gz9^X@0XG_Ves2hBs~VKZo1l|8e<|4_<@jtP_S^C_igpzRT}=7gj??j;Riwa)Ch zCnl_Z)nTDM2OXm7-x-i&PVJ}WQB&M?l+V!SK(NYmn}aCY(ip5wdbKNIbyF(sV#1EW zBO>EbtfEF;y4}wvUOEunPh1#+hGs!hwA${H zDXp}N8$cD%Z(lZ1O@h+1qKF9wO5Ba)Za&oS`pT=XT`}GGsbRNS`FE}06Hsu~yk;Va zB&kKi{KeS*;mJB(#?x=d@1#?IG1v&KmtOvPRD+qtq&!r_6j-}NIB8HYHw~o}=geVjOiVS+7AhxqK?MGQvEOfAz0VV*xz`inGjeJ!Pzh*bnlUu9hHyVBHPN!>y@g*wsTp$=;`V*rm$rj zTR5buoA4P3ecu?zU2o)Gcm2bO{aCUieCGUHl^fP69rFwGu6fVAZ$2;|nzmJJiSVDO60-!+eGwxE+nrNB2D>V zLMj;DEg}6wHs1(L9B#~nIz6mKT2HH&^*I7md+k zTokXVT0o-+E^4@iaZwTYu{O0lhpxO8hgT9={%s?aJhf`q8DT|QG>^1KSy*YIm{A(Q zOLDc>7MeO^E)LQQsS-?wz;ieLUbMnGYDz1|t>(k$W`zHV1(HP7IReg} zcp)*AyG-B-)FmYslE+(NseWCZU`@0f2sAFZ>On{2sz{U+vOPO{Uw@!iS~D!^Pv=0;xg0TSQH0-p9P&gOd7L}rZjDuL586`4!JFI#@}Cu! z#t3~E&Sd(|KVrYRIENa}v}Rcn0uJ69O9Pl~N#SW?MDYEi`g3}2MBeK}czZh)bZ}7Wk zzx>L%2I2YwcPs9`pBA1oF)jr_yThW%lt;p zv^iq+D1_8#RA1yxBa1n~?$NGPim0{D9af%vFA`rrDr!@fNTn2SJ2i#cE3B24T&u|` zSPgaJc*d6i!M2coOAA?yj>MhdTJ(CTn@t#6*lG9>aVpH-`^C4OY){uKmW4g}X*11&e^n{&Mrq5?wB zXTr?rI0G69T5k7d;zHt5>}2_Rkm>!Fk7P#XP!enlO;5$^ypPJy5^2^i>~!%OCgMXG ztaJ!$u$=BY62e8$YNI8p4^7zQYYpi!p|{D}Yzg@Nkt**);4fc3HZ3t7h=6LZ!;k-w z>ivY*=xzArmt#F9Yz@8+KXyZ+zKq{Np|t0r!#a=p;u;u3Z+?~J?r~PD17x^@l=vCn zrs-s5>Dk8Y+dDvpE0pSb>h^eY=fK8B+*7VVs_Uui%gMI>=hE3ze}P|rMO1U%DY3SY z5sg-Dv9?;<=>0})EL?o002<6zhmfIUyA`jbv+K8GSo~FT@rtkvIGHZTx1omJ39hSJ`M#3tb^7e>#&u;q2o_YHq_-SZKEtV zYbR2GMd(Lihvl7jk5~e)s_n2DPdmWvO~BuuYlFt$4WZr4W1m|`rlXKSW}fkdFr6XqSi1jBg>*lFr$muk5nG8YddtHhxpq|Ut73->X z&AM*gux?r|S3sIR`Sp!8#~7{g&}hFX>$Zj1*pPDQ3g3S;lQLmrGEdl)$m&C8jvq*4 zVWCu6$SkjSPsS7*lC}{|I(2+}Z942fgTH|9K1$(N4T6hber1>Z$NAc+=b(dU=~RN{ zPk)6szRaFiaemv?Q6qduM(V=o7Z01rjr3mO;3|d!M*dVPeouFM zFAGwhlTkZ7WL%67Wn+3W5~-fD=KkYY&+>7yF{jy>)3V;)H`s68kd2wg)^MJzhx$$S zTQ_C%+0Mpn7x(S%A36!F*SKtbuCsl=F5CBm?7I#Mf6JKFb4(JQ&p}zA^%C~&CE|VW zW}l+_ZeJ4zux)!K?b{t1C+jtSf&JD6**foFYq&$ahTZLBezq5~HC)DiYnevtRouM= zT+iwYR`sX7)kEFH@9%E!WAr5ZWc}c~vpdyY_`Pah`yRWSdt~1-iJgZ@vh%Q(jae(Z z<5$=|UNI$ODp?=*O5uCdV*6l#_4JnQ^%{0Q*U0Y60k+o%WOs7~d(R5t7{#`ausbzd zwmwVP`Ye%++04dlmW^4!=CeRH=gI6nlV$H&&d%p@*Ik-mJ;mQX#Qu!!#az|ZWX16H zJ(1~XRhJEQhq?-eupcL$%q%vUS+Wfs$i@s5UmV-M%`VO~3GY$pEK{IoB}B#cQKpD5 z$prEmCXmOAh0oB?e0Fh`%MMQub})L#42;e6x9rJUuq|Y4*0<#nXY(o-f8Ur@RcsAzh#4)9*w@&;-4Tx|wkz28mC4qiyN|wSm3VLa*_BLbRLTzi z19r(C$S&eKHfEjdB5r5GW4lZt++=HbQ`i|!`C=C<@)h%j_OutW>wHlBRntNbSaXyI z;#D5xFy}%GWT$Z&6ED+bYGnocsui-;nZ(9S67OiSJ%a6ef7yG6u=fm+?N${#VO8Rt z=wpAtPR0i^lVcYfvr9Net+db5Q)o|=t>ID!&#UY-_hBa1T0NKzNj-e~O+KF7X5sZ| z#35fr$ET!%H5N<}PI+qRGP^jF#h=x)_zK&KyTaoG>yJ%KW`as)?Ji*#XNeZfbQPdl zXO-(CjuBSI*$y&dU7`~1K=XZgGDh0p(>K|?KrWKcNv!3sle$UrKAZ@QWYiz`s-MC|<`d#WAr!7-?^1mwk%x zWN7{E!?o#{PlRL4E8Rb0Cv0Y#d5mWqm3f$+<>gouj<=V{3h8?tI1wbY%3dS&?ypE?Cb026Q?xLLbQe^Kl3C3z!0TONILK0HE3TpErW@+Y^b|L zU8?r?4e&VwntU4KacEZROR&EcmV=!jcc8D#*E0Tt2!z)GvIiybm^tGoRrIN@!N(wb za3Vj3X4x6?hxop%Zo7tks)A|G%Mln=ER7?0yYt<>)o$fQOz0<2{eqAdjRA)G?keH{ zx>y|!XF_H$+@ z4`r6yr%btgDzqfc9`>y?#(6G#&tSGbgM~pA#kP+#KWm=M7H1BxEp~W`Z69J|4vF`x z*nYs~BM!hS)9sn;+lR^4c`#e&!7|JCEE{uHwii2?4>nai=A+_O4hvam45$l2dzlg+ zBlCF&u`wIPb3U?aHSzp<;!VwKm~E9@c_C zRSgI!EI-l)(#w4;2VJVUD^yTksUcXye6b9*i&0EO9Xp;3SU^9cec0T+qjc`#U2~3@ zy9*(8xKD~y;PYe6+@)N!Ptny+TX9?%wN_{=wUG87Z^S?xJX9wGYLD=Z^o{b3_MKNR zs25cMzm5P?f^*a<))=1^UVZei9BY!>^Vri4EPDSWP^-e{1x@J)St3q9jG-H<+&sF)Ny<_HspEZ&W1i;^ z<@I$R0)P2xD~%FHMZO1bdXe*e*}u-)0!7eDob5~)-1{#}9?B(m$_V`3O+fh1-FH;e zJc>YXa$EV=STF1|B-FxdeXF^}MU0}P9b`=KP4sy_gU$Q?1m63qxP~7=GNvQebao?eU-i$zL~yRz92-3up6J&6U~qA}dEBjofp67e4q zsEYn$@=NIyFoZ@*M)TLI;`9AE ztZ#z6Aq(cIt2Wqv7TBn7&5f8YXgAZL))6w#H&>Y#gTL}?)D!~qeO$Tksu?>JN1<0! zH+eK)`O-cCbiOsWe%}kW$|70q69DIn9*SE%V3lMHoUlq`54{I(ewAP`Z$sy0_?El7 z(Id(!6btMG*(jDEwvjH8l_&RUj{41SKr6AlO%%n~39^O88QzI?>X9jtd|J#plOQ15 ze>kUD-jWTgb#&WO#^dk@*)Vb?KB1ObWdb z>9Y7MmFHq*<2;4sa_uZA6w zXbt#_Yn52FBOQopXq(__$M3AVOzoSugD=7JHFIbZE?@f*Je_d!g+x7|-^0)UkrKWd z8vU=qj~>0TN1`b2H{hjz)TXY1HkhA?9=b<1_rC4aLmVeADrtHG3O?{H_AT)(^)2%) z_r>Ycj_kL5NV%(j==;bwOI?hpg!r-^VdW2R+;Hh4L@2Bf8=^HALP? zx$Tf{c2yCTZXJTZfbTv^=p#}4%V8vWnkw(BS9Z*EnktS{?F}8*9-i52I3D~rcsk+w z(S$2e#GQ&?o^ftql7n%$BV-9b7G1>S%Ju;}G{=a-)DJssm%iyzc|mD|u zdLx(%bw9OSd|9W+r4wWaDN z-)3LPV3yxJN7)|=Z8V35Asx|-IZCqmjD@^a>6~%b`mi#cAtcxu8gYFo(bVVGU372I zN@rN&Nhmz+UMVR)1I;WqRvx{@x7Ams57Gzg5sl(1(oN#%q$5`DhE}1Tbm6NajU7fR ztY#8ttk5rutD(z#m9sXTt93$~-4fDH z>3nJWM_ktwz5#Rx$XcQ0%f9EsY3q(k8K-|q44^~G{| z6B|@fyDC}<_RhP16x<;6i(+yq)Y+^Jq_2xZE``|7k;8Q=P{F9DBI)0C24-+ZMd&h^{9fa0{BXRga0^$Uo{|oV&w~0qcYt1A_ zL+mtkt={p75#V4jjZ7Ua+LKVYt_Jc5ZAiRscc^`A9wdL_Tc@s9H>lB~|CqE!A1oF& zc4!!bF$~jU#_R`5Kf~SI9ytV!!MEQ>e-5od^eYxo*JDv_-sbdV-Ftf6gjXF*deJRDt_gedqr`M|4|oxmD2mVKMB+flH53Oz z>(&MG5BU-!X2CpO*--be@44r{b;ReG1@`*9+miDvo$Vr2?9L-c9Zo5+OiM_MvI~6b zJL(f9Lipg=lTdh8fA0Bn`gZla9^Z^Z#-`k4y7)=4!(P%J-R% zoyox(3+VAo{soY_QQf2-^NHAJFj;vWr{;eey(Q+*5HfpY=s&4B)HRqZdfP|J@{-HrScO7SSd+WU z7>bK*n$V86Awhs^Y&A9auIz5}aV0%UKashIbyl95#-jEADWgiMRMtNDIOy>F9)nDQ z^Mxs{?vyB(XH*$XK|ThVr{X_Wag*$vR>40eaN6)=2lkVyi6Ncy_2ScT+3QHC!q(j$ zqUTet-%2PIIr1-|AvCL4p@ug_5GgR*?%yzsHF|9z+q=fIrfq3J_vH7%c94#PDR=@? zw1XRa)^S8r#Ni#iY+p}yUQ@#I-W@bB$k3y|ql11+SKHIG9%XB*_Sg>3y194X`L{ls z_6iH`cY$Wnyhu5T2=Z!fN8wK)@!t`$gi$pVV^~MV&flmwB2-V+q$P^AR(UXNtLcaZ zCD>A9_gJNDPKC)G!nmE;th>tI_ixeAp&jAJJ>xs;JLfy^ixPER}S@chUE`@3M9&7THBgr#)mYwU$|zeOG+%`>y(i8Nq9UXw;4jx0BN+A>#ZXZZ5a_h61A zB31yYpR1SD%jy;Nsv7No@fgD z4)HX1Qj-7Wbu0RGWmQ|^-9@X|9x@{(Hl7x_m_l2SKE)A`+(_j@{?Cngb6H`|7MP%|K)+X*5uOJrsj(n(p zR9mX{g?`W<20$qc1aTC*4K$5h+cLmXi-jQgC_r;)DCQRw5!QE!wCqeM4YI(0a>PW9 zm6+5ZgCclTkWxtK(FR})g6f0`((eK?ZuGJ~&z+rlts#3^J7~Ht-ZQ1&o44&o8uyHw zCmcOf_JcL$`S7dJ0iH+ksp!4P zUdV(x(cb2cA*kY2s)~O;wP&B-vU0b`>5q`ivKGBD`vXO5(;;wIzo#$Jm+H&(<+^|d zG6l7dp!x@uA4Qra#u~52ehtaYPLMkoE-4XLLP;#v%x8`KIKN0S}^kw9+0k{;zr7tarN=7g&)yy4)+wOqRLI33tR9;$rr z{H<~QSWv5s#3z?M9k6>AG5!A(G#{o8S4XG`@_8ehCSEoL36V5_*MGZDkEQC+Uz1pW{8LDg@{PLkrqzDr4UdFQ z4g|-78`^_hgxs*t#mq1V1PIl}Cw*WowsiR{11-TPv~a zbm1iPifRW_teIfV0s(+p6!+S`elRN_9~_DOZB#d@tMrAsD^i8u*cf1nHCqxtk?N1X z?-@0!lqF9hep=l*Fc*+~qBe5}>+opooZ4)mvs9s@mLx6){01IIepb9uX`qKg$3$nb-Bj@)j z6^suc*5#NpG7*#nq3xx{_aClbTJ~4QA4pg$+r_X1mclY9x2iQPmpM-ZSPmb8{L{nN zWVUiobzs=As!Kzm(N}wv(}gJv|H4PWtO2B0av?c&@2E@L4R*={$7Mox1LwT?xoua1!OvzS-n33)*) z`2%>0Z?<1Tv%d)@LC{HT1}WVw24}=5+|zNm@^E~#cm33YrnSB%GhrY=S$g{_g8)@Op9*CL%DcOxVOc|vQR33YDiQxvbx>ecPs z_o<(#`_%)SydRu(nPNLGhyCs*)!CI3YSa_N`4xWF9Uv>ZzPKKc)2D*&LYu$^qT-y_ z&XLe+Bx*QYi=!rVyMGb$sD%+S6tSvyNvD{fGh= zCpJS3n>Q=bKOsrgCe z5E51yPYO^E6WWiY6lsIAcpWuFp`-KK}SN2ZURrNoe4>Y0o#AP50MWkQFPB7^$0}qwPG!f0)r?% zBX9P!ckJM|s6H-@fGq^>D_Y7{koGBDQf82uL>vo8xw;IfvsUF51+vWLK{XWOj1FYk zayD=+DXA=C`r%0^6eQQAcXzY4W?WZh!H8YqPY)kv7U*FTJDgJt5sVs1bBXtaC}dhe z+F1R4eVksQkJl&YZnF$oFNKhL6lAeu^zNoms|vWLk?7F2h(({0>IOf9`Yy@HTePhq z-3dQA2FKw9oP-&c0}I35v;PlX5K%8*CfCR*aQaY*`BdnpcvpM5u{+_7I}Mykg6og+ zAbE)zD>*v@XJHPROCrs#`AtZqi*a$1O*4JACazmS`Xe0zXEdi?xZ1glT1;gGUwQP` zR?tM$S6a#&BklCun_OLetDN(^vZ3xK{ol{Qe9b|;fD5Ldfwo7?n%b{;XC?0QOXD$x*p2jZatjL%N-M$%qdP;6nB@n9ir1=VdpSyxmEe!y z>1PvdkC-S^#^aDD!r_i{bKY3f$fThgEk}@`Jv3z{xRQ_fMA^NgcYr>m$*|c?XI$K- zJ1XBL*zCHcQ%|ziioXjl=IVqNbCb5`&|<%1F>{1Jfa6TEQHRbmg7VkHvk((z;8~nj z#*OG^I%d%Z(yHr#B~L6+Pi!r&OBW`Bb#x=kaU%u^u)O=qH|U_>I7y#8X+GvF*9YF$ z9g~g%;ys@!CWi)Z&>Y*|At5FM?1^@UT-oVF(E5xeL-!EJ*_15Pt=qKSNvOW0XkQuG z?_>6=Ux#1iF4{MI%TIob4l;w!LOU*V3RenuM7BqkFowr_sO=ylyyDUJ@>2hu zSCwALZiMpww!4~%*;<~}OacPK)jE#3@P?Vx_xQ!ZFMyTV7)X~HyL<#a4dAV}Y48>Y@d_dJ z4txRP7UM|6WJYrzsQ4G>=kEdf+o((DX1^z z16FOTq0~`$iZhC<)70W1(5iVGSq5t(i5oR>W!n}Sz5KoX+^S*UxkYuQ&2;NS-5of`xiL8v#qjxO zy0gjlkAQRLSm%uM#swqJLghmWMwP}SJYVSZ47Baze}p_0Coa+<5G9oCZT2yHnIaf4 zhkN9s=+RB(kakkafKHHG;_vJ4=kM>2!()e{LGcq*Jaji!Ej7s!R!i}v5zV9|d2XX6 za-jzJOa0YF84lflyEjX0Hw)W0F-1Ml|MJfx0UMgUcByHX!jz?ATSL0D3`~@i;vRL_SSOBfWd zB!X{@VGKt4T?Rh35>s%lLSha#h1#S1TYWx}%Q3_OE_&TNR@UN)W>_>}a z_6pLfC^Xp^d~dx?yW*gQjV=aDa?zKdnw$;^6+4*+>Qzu4&5eCi^`la<#OJNu&{+L^ArG%c3_zm!IQTt{xr# z29+0mBQ~9+NFHhIG=PEn2p9=d{8Rm|RHhfK8Vi7U#963iNpom4&A(J#VXQPl#&mzB ze}>EgWc$|tkt4Ufyj-%lDT5siV?&S zI(;0PRTsp?LjRt43+`l6SB2K7Ic^3DDt2O29$phH&HQYtUGAJ<^C3`RUmomhm~>-GIJdX+CNL(`Yv6BLk6tSI9K6fX<)VZRXwwt@7R zmYwN$(q^)AiR4s-j6yso%Ri?{PX<_ z{0sfE%7SLndL4I{*We<*o4gM>Rc+$C1m=d--=QB45(8&p#eU|l?kDc@7OnS^9LMQO z16Z%jQZ6VLm0$kK)v-q^7=PP$_;QAN=}UrLApffAVg?uAQsk2K zsBaDFu|532zt}H|F7dl41dk!0K~#wF`jxKtX05oih-t58QlJ@jWm-X6ZeIQnQ%X+h zj>k^YCTml)soFGcx>l*p&^)R_r2F>cnfK@YoEg9ogJ6kY>bT%p#PP`autp{%E%h(+ zpM`0b>~^L@t(W-5QvBclp)?EXfh0$by2}Z#3vfgH%&Oz#JSV*4mAt_cl|pm$x$Y32 z%h8wlMhFqs2(0D)5B(8e0NXy_f)1=A549KZRN6!GSI=WTJ$%7Ds6R{_t_i07%H+L8 z%m%H#c>DDo4svO}EQh~%)ftfe`fjaCi#Iq^0DePw3!yMnMjjnPTkeK7Mbcv*`MF*v zI7C%_fmit5ehO*|z-cJn-$nE^ExZ}^l(JP6f@#;5i1iKQ4luEPAc9bhkz>M@((u83)ZaZ9>N%@pDd$PCtyIvTk5n40^oRVc(!d=oaZ6OGDd2_4H5%X9 zOqaTx1Y1?hr$~7HGr&<_^2xkzY#D8Vb|foywWZoIJ(v&mL%Qhv9&9Vj$f#$n^}EhUV-AVzDz*}EqG4919H@5(z5v%SUf^a4U*+NgcFqMMnyTxP z*7;*Gi|_BB)txxV{+dMT!PmzIpeT1^@V)-qSir+*RKE6`j+bLrmFUf|w(GoMKk%+fEBf1srccv*bw_I@c)6IZ7(YZK2_{H@j z=?Aw@l=24M|7MU9^Kio6!zRB2>dIfR&XBjkOw5ILw*g|F{Y=H#u!gryo`8Y_)QMtkekqbSl=Df2A?jX1q zwh0vxG=acfeY<~$KbC)U%?f!8j#r`JcJsX z^>+FNvvqO5$3orIF5>02+j`!*pR$vLTFLR82u#@~uDB8`<-S02As8fvA=`qMT;BVO~E`2FFKt<4+HHH*OBO!w* zCl_mf3|Y9PxSVdaIW#=xKkh%_cSicsC8S#- zIw6el$bsNV|0(}z{~7;T|3mYfKSCqsu1!sK@k;$v9POb8XN$GfI`6;WmmQoeL34Ke zyLphz412EboDY~v2qcD_qMepQsgXF8nU~KHqORbx3n(nA?&lq}1QhTjvWi&^4a2g) ziH2}l0=!k12L6kFXffDl)NQ&WiGb4XK21;n$H2ORzuy;!|TFvYs$w=Vl# z@Rf#OZqlp!oFeIyFAErAqE7tLJZz`6yYvoC7G2_&&^SVGE!!YOnA{qgT=DnO5>H>s zOd@RMdjVxD?fZ;V{=VWl_R{LP45uw&NOm z?KN*+vIYJBm)^s_#iH$$lY|TJ@piCS#`m<>(#Wm7Rt;~MljKlf4Sv(MYko;WrLBnl_k`zB&xiUPZ68}T)V-@l=qY6M zwt@8Pe$V0^{3Vj`;%HL3sECD=

3{C8Pf6Y{`=?uJ-1|cY#N44jH}eAfvm9>36U# zg_4fVq2UexP5&M33+MyG4#Pdicq z5eo^SlW^L=lLG*`6E$Q-Sl{GwkFG9=nS{L)rGi{$1YghO7CB$|@A~ig@B6P z1pG0An}}tBB9Wf`57AyXg<8>QG4(D`T}3w3ouJ_w5v<|j*(NBln+nb5%#E431eySUp_Q{ss%)00qVUHA+ zR$15lBWCpeF9-7F$M8&y2ID-5;R4Dmccp9``U~FKd!uYkCL1MgoqhlafaeF8irBwd0C$(QRL7Y&XxnS&|X{uFgRij z3bLC)17wrPNHq%~^}hZCfD4QaSZ*6UWay1qJR}cP=M^w2T8WSHPVVem_C4s&HKF5w3#Lc?gRuzz4cpfvCaarSdTH{8{(8s#!GAt_4Edkh8!Vy)8opf@WtoS(mwqK|iy zC+9LuR{38QzI&L{aNJ`;bsL&%9>P4LxzB!1|eC4XA9l9YG>c!%du}Eg*ausVm zCUs>AMX7Z8&HjoJtkj?f3ATewr!lt0+G>po%(LdBmK5B*YznnT2RNKuZqH*+J1T^Q zY6MjhR_DDKO(8|0VehONQ2{_5L<0VI2eP(#b z48okonrAq=fJdf^zCRm+lNAQ86jvxC-giXd+sfTqoxf42*BKW_6YkqNQ5AZIZT zlE((#54incWGy`jg@OiPQtgcb-I4x)vM>_r8zl9DhLhoC!^d#!-*MLxN%80|&sTAa z6yy|VLtX0hGn>^_+27C&RHd?)KM~b|3?~bKvzZ5v>xh@jMloq|B<=7QtC+vIiUfZM z-@r1V#D(UGBMXr0od$5npiTd=jg=0y)~XXVX{II@l35*tLvkWNvnkYiK)7u{{gC7d zn)w^=KS!6KyDa25q5nO0r0$XU#2^SeQN6W46vLIt!PQ@^OT{Ku7x)9x`~{l zalG(923b9fg^2!yv5EXzyLiwdwPdB|R?D zW(}ct_>nvw6SZUTUY{%D3h3Cnvd%^gw+)(49J<>3c=4QC1=3G4>)qPyD1S6P*K>Jq zmiM&ByLvyP6H*rGQ5k)2doLq|yg!=N%RlYiQ5uOu&#@>H?J)yXpDbi#=VePL$opKm zq+C|6P%8h=w7r_><-BK_7omfmqr2No^S93p_TqUw+CuueIGEE?qems<^Quoddfa44 zqQ>4Y#e4u~)K}I-rbdz9$D!G2B~i{olqhWtO`@D+(w2)Tb=DfvBkct|5(50}nnR%pJTbO=leOb$>&Lu}bLg9c$gbV^`qU|L{$pfWHcK>K&{4pXK8 z&`5e5pocirSln2LAL1^U;OCnKb@|T{lzc>D>@29#_F?ze92!J8i3OKgu;DS`5M{*2 zf22bSC3q=y-E~HpmL(~qLDGlA(&Ag0#z%nAhq@PwcY$3AHf-cI=2#<*?=L$?PtcpF zat3Kle%7w0VBBjv_ul=gI>L(!@h}jos8$2V}hU*|b-jXPQ3w3#NkKtX9;a+hWqCc?n~I&YwWH| z%-p*q8FcCXN;YVvWYG4>qnM#HQqm*rWYyp32mMvj0BrYp_G9OXth?8;*{{mH{#=x1kuKbnG=)YwM&dLwypwNE>o1%#F7&7qq8sb>#CIGMhc4i=U zIbVC&d%g2=3bv~4uo+KtBE`AWh?@%k{7j>A<4nVGPO_nHw5rf4-Z6$HJ$xGJ9IfFn z{p_4%ZjyY}o;8_N4%nblMUL5F0C z&-t{^X0!V;TXH(rRs6sqkTY<5x%Fu``s~kktZ4>TW>$IH-76b_$uQSDt~K5IaCRXV zwBXOzp`;V9zFd(|!PX8(|rbEc}g;K79t zhz5-4rllP{q_5Eqnyzs@nT)Pw(`5f(0t}`;ru2{!l~~GF!79A{PEn?=9v^~B-OJn` zyFYO+cgs_Q){t|?R`D0HU>G_eQt9UVscEk}yIzEhg$T7xqdJ|kVx9Zsu~(TEjWYB2 z%j*7%sGwq{mNXiCH3Rzz^nL`1p1G!-L{DcZy6MY=XNx*pts%!kAf}4@76y@4w-LXh zP>Os-P{^b#m!*#(DsJ(aijC(VLa1dxof4ECbL3h~3dOhv;GSnn&47~#oe3T>-A3OL z@x7Yya7(`7#wU_5enpW1Oj)qiw?!s|Hg6vp7H};HtJ-N%i@$n1(oUQ<`*F{2Fc5dS zjs$9Hs7rzvHBbUs^GzAlBvWmnKOfj|=PzW}t<(gL2j4+bB&gM26q9H4+1JxtA6L;yG=5;LSV`Sxhp4qXL zG{{+*4kde98@aQBzJ0+O3n5!uK*M9sW%@!B>-ktQ3BNG6ODAh*%nQz0eDgnxU(0Hlc z;ph#?a2FEjz0_@o&&FCqlQa5ReT??LHdY&_jTaqU_~$Bvddtj@%}-3hTxX5Gx%N0I z$0nc+5^o~6^KPFv$%o4dcY;Exl?^@(!DCuO4kLRZF=j$BOXl!TqznI?SJNEk$MHwa#3BfHfI33Kax*Qq6z}9uL75acD_w3j>d|54ToIbXM2feF*ikK@ z(Fm8i0L@1uU7=*8=YtWztS+Ys zU<+up%8Z$p4a?i^s==dNghD()r>hi~MeK@b>Kbq5iUE*H%Et_PIhE@9>^ zHUz^TjL|89tdUM+8(;Vq`^aNm)_+J@c>;kf-?c)|(E7;*ksH27_I| zkgiqd44v=r+MEaeYA?3}v|KV_jFZV@71n#so8!mQ3fVi#&H6(3pOK>~R2S0^4yEc> zk$+BFHWe%FVwMGJW9=T!CYgmHP@L< z2g@LiZ2Qev0b@*Hl-nmI_b?>D1XzstTiU-eEtvpCH$XWO}?;9~pyYsk$cCnJ_%mG(lqaR+c zMVgjX#`(Lg?D0K&?b64DWKvY4eTVpD+IgiMb+)>O8BEEvtIB zghpa+OGV1i=t|PUtx5o1NqE^FJ9jN@(GJo`jTELMVW5374+2r2t=d^z$UWi27(2d1 zgJP==kl$Y)pwrAkQO?A?;K|M@lJYwWBRWCh51!r(>8L55XnA#I1!A$XYx0%vK zSiS|$JU*9ffw7+`_tPr}sn+JdgxjXcB8`2{K`Z8kn6i6IXq3X|v}h?_ALT_|Gxh^^ z*LgVQmrUBJx_^E{E`o4;L&ir1?_d2Dvm-Nk=aZz^ouE*p|D``okuX0vO8Qo@x~g#F z)~E_{(2wE+I*l`~+nCj}fQe$-Kx5tj7#Zo_(XP%u;w2}T#boue|S-^(<&n>$RxlFF5%uJXkmq+6RX==bE^{JZ>+Nzs2H&-O!Af1HOOiTyzo zM&*II)7)j!`52FYAh&2Z6oj zK9iSZ(`yK(J1^W0@>C=-mGFmZ!^e#gwQ$*A9-)qTifoUq3}V1DLK+YwDCor!n-V#r!&$c%0IGDe=bkt1|m8D!t_-1QtZ z51EfVk3F9PD-wZ|oNx)2iO@VF7=j%WwX=aEcp zA$O_Hs_=>xk!X2@d_YTX(hk5W$Zs$*M985lFGBQvEFE!Cc|fEwG~DG}rXMknnxC8f zVF0kiV8JUzH3JsR|NKA5JpJ$ZBGl!WxexZk0XPVUAZ$cqS}mET>5u<+FXG&$43_0rcPDQpjFrd~AGTEH`S+<0k7}A+VX{ zDygWjTLM{Cu92=$F7!F^fEQ@ls4Xe-hEQAeIcRg(cf!1@-_s9iDUrfq?&0nc?kcyu z3~59oj*aBIwZ*b7JD!5EWOY2(Ir zRPS9=K}!No3P0#@k4C|kT7obRihlU(P*C`{LJ)_oh;@D)Jq5+ciB1V|{O+}TUDFkNt z;!Dp3bCcAv$(|UDwyZ8ta^J}4IRz3*h;ngL00LCnQhWKhJNBIP9E0p^#Tbyw_{k`Z z;jj@8^+)=4ZHGpig?L@`{*ctXjY)M6d~%$;_=f7RY91+p5$bRGN<_+_vdBgjqX?1LexTxxqhh8_|5=6Y zB)XjCl)On3oRX1ycq?e!+aa#IDI}U_U-KBrWi7m^u$$RyDm()%ZkbOYa>Bp6WO<%- zOi;lg%hXPvi2YT<3N>_h(Mf@Xu#)S%CE=cKoo{B-=RupaLbhRZC55?EI>JAI7q;uX z;*Cn(B27FmcBIti!=iTlZF8#&8#o{;XM4z}TKAT0qNtZE&iG62v7Up@LryHbkM38R zeF%1GQfE4Z`s@P5(_J%M82~U*UaVY3{?=p6=D(6b_+M$wn$e)i1#^*k$GmIaGc9Eu zWE0T_n%p-hxF)))U6WjsNm|JSR0w$50va-PtTfITEuj%B0wBoL^9y<|m=;c;x2KP% zmq*CS3Lp?)!Z$cJI#Q9Lw&~;^x_s@?q{lpp?W5yQP)LGHm~nO<~3uxrCE7fO!ucy^9oP1&4(3^xMfe9Dgvi-I?U2Io9GTsvJ6e>$g| z0toGMBA*k_TCMM_a$4C-!kaj$dO9Q#fUYU{qkaw#ff%OB4e3`S`c0;yKmRZOgq8m8 z{vQ6GegXBUvN}YUM0-*yS)OSucgYm4fGj zjx&lNOj|ni6(FYyJ9iCKKg>4z`;DK3jXwIYJrSnPTmYc~e%z2iHCyEvd*{m32efhh z|I!22AO*%Vjx1t+42T35+2vB)bv+~WaJ~W0iTXOC zqFx%wE~F*e%HD|@54pxn9opE}u=-xqMTfoAtRDp^ts!TSUksR)c9vz3J=nicD&mMD zYn9odw}-av*pLV-$!06AC1@Si#eT~Yr#+2Hm+1^uxaydPN!hy>6+cKj(MX7J6InF8*Mj7?^{RcMERBNEyQlDrxC(>+gs44 z?3hT8onVi{zB=+WeYDdF@M)@zM7yYbW>*6BxW=AJ^j{w8Jh-rx-rQD|Xy?>Olp0<| zbursU{%NI3pLXPDo!mu7ff#h6{mbgKTiHWTqnGj5-bP}hK*Y?Z;IHy?8q{c|M9UxV zXpEx6RDANuE1&OO7ocms?GlP+O;;#MC_H2^15n9Axu*Bj#&6z9*YwC=!;Prtow2aN z=a-H<>CH9(S42zv<>vRUZrs*@+n`xi5FZ%Hncs5HG#;&_TJcdUlW#rYZA0nc!mYl! zLKA|IX;P}~Cv__ukm2!4$1fr`|EuJ4QdS1{oPyuAoWOw^a?LDX4w}KMq2Z+oxkFkP zXLb}Gai07fugjnz#;rx)WQc5{O%Mv$g#5ZK)7){ z+@f6!Nw|d+a^*|Wyx3j3pxnAQGC{>lC+ei>-E39k}@9r)Wrp2(8Hy=QbzCn)Nx_tRhg<|`c3N^A@8 zEpu!7aRJbFNJ_-;35<p!EBd`;`aF8k*V3RiCd?+{ zj03_sl&(;+F2nRi0EcE{E6dand*#!s z#k8;zI)NDypb)5~GTaASm#iVpY@WgK@0|c&jRNkHuM6TpM4gH;XTT!{k{9!=6;9V z$6w;pX_5bmW3hjUf2lvU(DT^o2H@^)SaT7rmH80b>p>|7T4P#5qhl~yf z7NC{380z)}$?lE1P6eVAjmKGiZM?=X(MuuQa%hb2lArizGj(dA#=G1vokzwh&+WeV zybN5rF|W>n7ZhcMMP^irw_~*EsL(AvYJWnW@<+;}R$ise$G24UG zjM0o~1vxZrSm9sk|H3$DoHtZt*6;uCxJ+kMb91cizmC@2CMEFLi``4a#X=T^`U4*r zN=q#SpUMGal|SZBfDOHasy(qm)gHRtWuPXDqPIK<^mX$r41NFFVy4W{7IM#|xh7J% zXrEq++PLf5w1x`2HO@%o8U0KF)LG-*>BdsX*yX(9?RQceva0+n6IsJ?v2vID{jvjV zeoYSib9~ttZL+qGqC1;ad>LSVDk;5SXq&vHgV&-W2oH1%JSrTW zxwIRq^D6z~_)eS6!#H-VKI&ib`Y*9X@I|2UVlG)4AX60^Fsk+Ys5 zztuY26-wj5d}c)$F2YWzE3MP1=tU~fLV(ROWxpO5@N)o$xHtNf?F&(ZvLJY6XKnIdb_>lM6t@rvCcmg%aYw5vXe`^p z){Yj1<#`aeY+8_=`|bzs1)4AuEwcY&eTgo~VbHNHprJ6DX!0TLFo|NO{*$!ewHg1ZtHTObn|KwnH!$;8DXzAW}JQ`I(=c-$Er-J%AA~Mos5h z`DipPnf-rqi9Tz;Pu+zGTE;Zx)R6-C$mWK5^)iHp;@25OTd?UjdN!$m^l*3P_1lf68=_t+haixVY~H4|pRY^d=p@UVjtA)5;!Y&EL)`S<(jvkLkHblmVu z01{}o`Mf9(w0+eA@y%wr;CHSD;6C6_%uq=SN&Tk(ZNgUa6t7|+40^Ts5VWcgi~URd z2mK1vY=5Ur$OXOFCVX2MU#3(&v-yz7Mwu2Z7OiXr*^K?#?W>bWZK)!DgQ*%<;`R{~ z>mQ)~A^#NbXpObx7^$V2?W%@55}B@N0bxD+AU3dq1_O2@1SbxK5IXF~B^(?s*vV+{ zrl9sR8pF@DBJZFO^-2FJ|7q^q zp8YnHdF9-xzefAGz zwmQR&ST!}I=9&o7&id66lkKu=jH>=K`947#g)2O23L)@?|D1n=ljs6NVvZ*MlbWW2 zbZ|V{i~l3>UI}E4(5ocWO|-+dNg`WQtN>hi)gqCIK_zt)(@t#(h|u(c$}np73V+wm zUv5d>T7AjiCk|>&{@yVD8aS0=k9P4)ca1d(vg;gpY8>6imUBgKS^e?qye{M?OXYHA z$Urm~%=3N*NrYp&CSYFhU-Vz{U-n<|U-c`rE;%9^HN@a0!4q83Ykr0+ikg^qRevnm zY?ThVm^1O#QLa@2St$x1KeP}bMHIl6^3ze>h*`wZ-c$@(m-XxZ8~&UAFa5XtxBVA= zGU!)Z$X%DFjfr;Q63DW?X&T-AMl4H5{?Rzgfj1RMt^x=xa1B79@2E36q?|61R&#}t z!QNEtY&Ma}gTV`l;7#m5m`z4(J<1ybaD3_S7*tZJl3XWrQdDvtTx~h|`FWRyT%~!d z3YmZnmGjVgCV?C9O4mt~FpPvj52cb|KoKq?{}E^c6^Hstd>k~j{+AFBIHr@;d~jVz z8IruZVb6TJO#DYY@x0@I;N*m*1ZB4DyZ-UM3BD9*m}P%+%UC{y?)h=6f{z`WL48^Q zM1rI!R}m~V*SD@A@1Z75K^x135H+(wf|?mxKcw`fabNyBIpo(;`*X*&iOt9%b>xR^ z8b_4(V*W(4lh;#mYvWduG%(U2{4BJrbar?4aQ1Zea`txiakB0}GEChUW^_b^@#&Mu zzc@dO9`KP+;g(b)Mn4w)goYF66f!cA&s^k~&s@^q+8H@yl8t_(T(Tw_J4=|lv9r=+ zeEp(xizt6udW@rctkc1$Zy3K>fvXg`F;Nl569bA6L#GFd zXI}XbO7T!0_)}s(d=07_luq{I8cv(zjuVcPj#Ca>C#&Yr@S*ZxRHEt#g-7(G`saEI zH>&VL zDpv?nNymLThS3$NtF$J(QvHxxDZQn{&7RmqM1o^K2>%s+dB^d@PJfyD016|TA4v-T z-IJ3(qR;yVYiML^Dukc_9VsQi&Jq`OkQFc2vAS|Og$VsHbK}Xby?4LE+*B7Rmfe&j z;z7ll%#D!46IT9$yuE}~(h--W&?LGv^6bv$&|s;~wguUhM0CQH4z)8={)xP^H^c2A z57`q-{F6FAh)Qw8Pj{Kgk$+3d&Aoye=UDoTTOe+md+d({*xNz)Y`N@S4VB%iafXW}f_mNwj`b%e z_M|`gv~<&t#8w5U%AZpfrhM~w7y#{GbN2 z_N;X?VNp}?|B#pNLV(pwy};2pQmraSKlqrgDnAIfhg?oqQ!>a_IUX;V;AZ+xs=3Fj ze=4>X|LM;j(Hp26U)JPHZJK4RE#zW1dcoKoFBuZPhR6R|kNfm~!io*TcdgexsxJDy zCvb!N&d2A=X&lxXauVr$G;ms%3jQOJu2#w8Euy~Ucs}*dhtO09uIBON5Zel3)G>4; z=>fJ)FHN4AtjF=#F&d;COH`c$b0}I9Wn8`+4W2T$IB>@rG>|;% zd1tQt(~WxDZEVbAZtFmkCvkLzY+=J7(c4`X;E8JB-^4ZO5Y(Ie^9Vck6!e-&$B}R%z0^cJv zrz*WJ(MjJ1!YZusfrZiV{@IKFKi%Q(yR>z3`=Ju&X6m5J-T>c#?*PDn0Kge3__F&B zeWdjC>ab#*Ks$LJ&UnqG&jG_cqCYNM6qI2rr`l@gegjV_iIHUby*HcZvacEH>7>8_c#x=4+IPp^~1az^*H7cnO^s8dnH$%6P3!xzzO?AQTNHtQ52TOrXyUt z&=2@(->LgQ$Ab11UKt-oFXY$9wr5;vVl*bR6?F;hD{9U0qAu2y1L-l<;1gk4h=i%mwN$bmMMJDXUKi52AO?>{DJ8oHQpQ(; zU#Pj!mo*@i0jr=YTmYa7nCHRk$;7wSj9wU19YH?~Q*yTo7n=n$+Ir$`Jl!=s6E*&j zb6nl#=(SMH?eK<;eMWu?uA2=*TANXr4*GOah?IT}o~MgNh_0Cg9{Uz=@lg30R2*4->6V90hz=kGTbP1?glF1(Yo`$CL3VNUG)rk(ny{S^ThOEx4?d*?fBA<@ zyRx)u?q*S6rheCc4QSZrVbohSHX&9qcD~T2GWnD0AK~~j{3A>R1@I&QQ7&7U91do> zLHT195?I2fHEoN%S|;l@?Y?>ul?W2ER7AJ;6eYml!6G1dgNT6}25)_32p2K%;1=vF zdU;a}PkNsFr2zsRU2cIzl_qJ(Hl2muo3Yt0ruEsyxl4Rq2?=Fw=aL^uloGxSr}I-6 zGFU!@{FhD8NLqr=bOC;Ozgd7;*_X6yEu<V`=8K05Q=qT(jm~?CD-r}*~=hP4FRh4 zvE|se!ddPJb4r7M`t$NBYqYNvQuexTSv7DEA$>7ybl4~=H_Z8=jy`*ab@q;294pcY z%8dU25EiaJmk#ohVJa}MA<7wLh)+iz42P@5+eAZS90CvWkgt>ISqrR_bfmpCdI`34 zA&qUEY|)*QyN_5xU&rS^wAe#)n~|1dvL?)=0mw@eOEc)1m;?2qh)_0A6FVmc)lV#m#}@56%i= zFX>C$W%w69rb4=a8gGdLa~6l1jiZnNNtRZoujzW=X^~T8xr2hxyo z-QgdC`_X`B{Uzr{FzeCN60hpH#o4XF6!_ zv0q5;5Qp|AX25Gr{yVgER#|~4VzeadV4^4XUn<0$t7R_@0WwdgDah?Xq-&&E!SYo9 zkMRlFSV?8L6~2gPg27>I9VbnsqPDJyY<9`EnIgd~jreB;+edJ|Z1@5C8x0(IZ^>-T zT09gM2-*G!OrC$p`u&HPk^hUY^wK6Md9?BO%*6oBWCam&&jM+c1cY(6%7f~v8=m`0!)tqNa~ec1Lc0|$%z2_@Obkg8w6Ywt>~c3!0V~u? z!Gvgavl3}{?sESm6!`pFI~j9-SNPB^8CGA9mNRFioS;DQ)SOpR;;nfwET4jz&TeGA z-DdMmA#{(jP%gKxnRjq>3gCZjmMK|UrY^HqeFk)tc_+y5esYlHc~)%ajMr`WqdTUd z2?#S0?!#c_B9#%1RDgacrxSSwD$b-np{SbPcX|hk5wNVFeiK$Ut^#7@^$7$c%KQWW z@{N~}8ORBYoSVbCCu6B&t75HWG$6$_%X9t!6m$SGy6@qU>e0AIahvp?uPAO(JhuOa zJh}wmn8+E4+T7{^w0A4s61;RqcAyKtnQ(8^ksr(l*?@ggBXT2mN z*(I;G@8=Tul}LA;KtM%$*xNy_xV_kVm@DV^xQMFmM@`OegNiCBcEu_Z!uwA18@cAh z%?UPy^x+3)Nt`$7>?u^upKMzY-QeBEfwzaEwfDqhTuUIGwT^#rfGWYJgP!Bt7Uo+v z*?fYuV$yfY#lun-XY2@<8q`s<4{~_K;<=nVii)Vsr&0hRVL7E*8z|bKftu6F*7o*u zYozc2TuPw|W(Ebg0I!tFUkvt&`V0FZ^7gRCLb_;eUJRq{8&Qk(91AdmNUVSuHiY$k zcCHip7bIxKzlMQIAqH15x^KR-QN%?QPlp4fcP4kLW~g_lg6x?=;f&EvubxU9jVFot z7~&z8tGLXdC|%nyRE2#@h-t{Mqb{LYxbvM_xqe2@COfM=eB-vm2%`FoA# zyH-o(E|yFjCJpaM4+|4*6i$=ComN8uN*J%oM7N^|QA)DMQ^g|&EmDSjo=@+&kzA?n zdeSjXox~*!sMD_6rg*FC?}_l6 zTioAguyMPXq+O@HF?A+Eqr7nspM){AiR7$GPyRn=>^+t5rPP#V4uzlTcv}4VwFO-) zlwEe7#-tvz4R4i{6UPc{Yp_zt?e8+V(3H~7K?@?K3F8&9MPCybpoz8$?;Q#I6%dW@wEtG5 z&^_~pdlWvimzqfD1EnFjYE6t83zv+d<6L1s8O%P=2sgp0A zbOTKS9?c3p_b2Dx4*h#LGB$34gS7jgi|PKy9Oji#m~13nhe%G9#VvOV{%0Z(-);!Oez2I zLd=WO6n_Q%=l>@;dPP@-)He}z*W(8vCF^$N&j!7YQcR5u82H4Oao1158U{V7^2J@T zyR5(Gl~FDK;Bkwzz3i1Lh{DP?3>M83#US)RXzh-~DTP^#dXSh)8WDtE{1(mml;RzZ zN|hB<0JLPJRHHz&D;jE@{a?^X)JSiBZMYhFo{TM=T5wds*F^f7oUlg5d-|}}>kmAQ z;9Y2RDGjV^_S4nWMIcd1+u6de+i_b-;TPRHI_f`XYm4 z7|ra^SINW!XKd}PT5XI&`NXAxdC9|bG4j6{Un$8?v9NYK?Msax@9A>0XUXvlQqqYd zWi9et&ud>xTi~vrgPoY#dkn`6L(z)1Mke#pD30D6p5^&K z!ibT*X0tt8f?Z83z;ekijhZWuh0T4iQj(tI^R`+%7FS!@lqz5<)@uR&eb0ni*)fCE zy2h`TShbZUD%go$go^7m2bJmsZ*Yor++!gEEedOOw6hKsx*rHF`i94rR)t|+Lk%mH zi8*YqpJ}`(vNl~i2OJomC(3E=UsnAoLS>O12YmpN=KYH#-J$<{s~IC0l^Nc7PJ3Cb z&j9phHD@TC8Qmw+!_9JdY?kn6Ojust}`uhWfaD{`bom3@6i7+|1f?iH((1AYTe#WM}Y7$ z2;)F6Ozi>~8>oVD;Jfh?s2po6WI|;v^#I+fi;t&-(J5hG!XuILlhwRI$1I`NY8!2V zl^G(VGQ{Kk8(|~+bM%!5(P2H~79$hj@8Mwb<*lX7Buyp#J)AW3Kxv`N1nfI|EHY8$u~~#c|n<{eubMHw5p0TsPY&?f;0w^0V~_r4e*w5pAk) z%eTq=XV&<|S}N5!x1*p5IR+v~-kX`iy284gcKELi)b@NS@nq( zWdS3W(ClsmYLoHv$nWwIW|O*0h@Z`2)GlUkuQ^wt&9~pX2&it^6~}^DyxT3gmrKf0R;vJ)kcC z678o5)ju;?W#M}5;_n-(Ve-^D5WZ8Ir#^y<;gbP}#%U9m%xp1~}u-nFjsUkmQ@ z?>5kIv4OXOm{LOeLg8;nZnQ^nzb{@M_aqMk-Sejro;-l`SagT1RT&g|=m8X<@K{^z zHTk!E78qkmN<54yFUhr*;dr&pdS>vn6g<)jG1zY!aK7Pbpb>%Ijdtw)wKq`#8UuCG z+T+^tKeCK7vvu+{TQjs~rF%`>7#?Y5+`w(**o>Fd+vfQOMD_i48!imR(Xk2OfdE(p znUbYbQL%*{L!Cl{0{YXZF30DX-oSXmC?WI)P%vr1rV9o%c}-aStz@&Z8?HTA9O_H5 z3LKKa0ij`~1J0F)7=Z+1&+Z7Wo!Sr5Yo_N58xijCUOgq=QOW^V_hlCL_7!Eh*F)DGCP=R@_XQ)Qsq#)7-c47=? zw_EI7%3R4g__&ZJ1crZ+b#ypQId^gpzrG?T_IO>Ss2;QI`hjKUc?32oUT}i%>F+7{ zLuRkr%F?C(4oXj6d`4=E3-X2#c8*`KJT%UwD@NI2$pS0;{R#c^{igv&0UH3*B69?E zL6@|_;DZrdd${xw62`4ejf}+m{n`xpZ+IVLba99NOw#gZnOe7qk1|d$+Ny%1mKQ9A z?783a^rY2`xG;J^>o@o6w?YthfC-e-=!;FXH>5>w_V3GwT)Y~RIHD_&zYy-70KVA? zR6@s|-u2N5ZQPs{bVql5pn4)w#nsDxbdnYfX;wJNPi1ziv=Jdmi9%f4{Df*eu;#|( zXt?9+hj-hy+ZMk430%1ilX_6!(^uF26!TEFQM&Fw5di!^4&lhlv9(S^_-Tx9#!q7R zH_0X*7d&#PQi)*#HyOXA@oqFr2E`$lJj)l7m;RropOlu#tCURC@*jHsH$IQXo8TYF z-WC+q-Q3Hs)&}LhZ!8vhkEC0scpo~YP6*L=H3d3-t1-^Q#E$4vSKlM( zis;8qD{gun&p4d7tR*GiWfZrc=^0jJ>VIV9s4W5tzpe0vtn#(BbErbHY|RmyR%)Rl zZnsaXn}>S$&%aCoLj6MJPRjbt26ArTV{Ibb;@D*_B8Mi{@vHL14Kli~qqnlqo1c9o z?Io~;6+(m5pT+{FeNi~}QK+q!*Koq7@!;OHDfW&_o{a?$5Op}Wu+G$NS;^iw&*kEs zKHhgWdJmb`zbWSx6loa{5uCtEnBcxNrg0uz_Y?8tV7d{`o@gz?pc*OxUd`;o?28s* z7(uTq*}z@y)qn`C2ET&OdsamLJJJN$Aq%7SS3*v~ zGQYYxYZCM!s$cEK90T}fNF71nYF2ntYBy=ZBx(V;LTIwXA4G_r>?AUzyseez{rCnG z(-d+ju5MSt3O@I2qna9(MMavcMd#V(g*0cw9x162c{HgG_b9jJbOyqv*RZ^+zh#cJ zktBp9N%AxCJ|x>pfSAG6)6K)p>rg&;%R!0O4I3ZK$ZmcOT#>z}fe3=0EfvEe*p#vp zWSAP4;KzGUvf}DDvs|ZaY7#nevkZsPJVX4!1#S(sLRb#l0w8aRL-uUzoAW z!~hahA-7~u`1nAg{0TFV<81z3Anjs)+7!GFy&2?3f*z^6b#~2Gv1KrY(*EaIbj}&- z_QC$z#vJ=2{Y{nWj+>N@)8+Mioq1ZGgXFZ>REYKY9RAq#$!nK0D&R6v%^#!ZUb=1Z zH%^U2n5}iPL8hc+ZW6rLpjNWoJPMfBn%B)PLy?19>yoH(F6y2e$gdEMwEqjSB zI!}4Zmgs(EyleCJp+X- z>cNyq=mVQO&4NB@Z-7-kpm*2!;nyiwSki3-w#Ic6F~9Nj^1)JkQ(^?hz;|@3Y*cC!B2_%KD%iZ`dt1|I9)GArR9XwTTI9+}B{!dZI5Gl0=JxhC12XPM<}NYFl^2~e=l1-3>@vn|NhhMPHI z9etY34^4X|_{V)9y$#)x7*ao(*+f0*cPcm=_L*heS2~^QPUP9;WN7Oe$$r6U*0Ixt zP&X9+{|VquDE!H^F{Z{BLAWMSgkFqv>VNhH!Sv`;(%Ft=NC=jYF|HWgF_Qt*joP56 zUah{740BNH82VK~ohs0}7}nys*jXG-!u})3w*>i0kCvdE82k4TeyvIX)gX;{rj4-1 z*l|E3=JWhIBX!rf8A5M6+$GVbr->7geCOcQsK)*uNv130qf?H5&Hgc-GWDdjwXvff zIp1Jms!Y0jO3)oJm2U5-5gkvKUYk%^FsJ*Lwb~Iq7ifj$d zz82(e;tfb9vx9z^UVNTw7DrHyQ-oS2Mv9zgAuV>a$8B^617+*O@&(SF?8Yl6Da{Xg z*V}jTa5?l#J}APdBu0^-{zCyRoCzi`xZW9^M!K{!-mJEAH5h-Io@Wj2q5M?F(JbMe z!C8|tKT|23ojb6U_e!}1=V-@CCs>!CDdsZR@hPw*2;4ie$BXNJA(iUTh}5q ziSFV=O3*`nZLSI`s&E{Y@6*<5qrlFzkXaF5<>sb5&cuu-rc7MBC{n>uf)|#KZL`=5 zeTkC>VO#rQ_iFdv_qcnX2gatkZ*dG2$S_E`SwnAy`<13TN2>yv^|4*a9Of&eJ6I{gxJ<*jTHebcLC(b&2t@-c{!jUEPePmEcv?4!ZW6WWm`w zQ<|TMu@nxE{iB#Q#!}15a-Bn{^!TiLgp5STD^SHNTdLR{A)a+7nDzZ}1B-d{r-h`n z*?F;QtCKT4w}%N}lI;$gwFZMN^?LU@?KJZUX5+<*lbi_%Hlm)77nK7mXU&EeyzqzZ zjp;oi9OEj06xiM4YN8apZ?__5%w+2WrH@!}!+=RFfWbRoVKWP1=d2TtTe;b?j-1cR z;Yf{SxqtPdbv5E8<*8YujQSWyWFPJ=9}Rd<9*Eh%WR?!9E89mQ{|nS`v}8w7;pbv<`$RE$^r#%60FR@!`-!7 zf4!@p#;6g;j)W--YA4QZf{X&c=?&qVSt|2_l3YGQj%S}z`*i`%Y7AB$nDBFdKT**C z>)gB0D>dis=KewHHEU4(sjU*id#&ndJEq)Zjh_G$>BbD2R5+Gq#MXYPOjFMMd^rGQ zF>0pBNG$iM>$HA}V6Gq;Y(o_+`WU>^dK}d=!JkH(zI!}czF`2CI3Wxq_`)p^fC^}) zSD3(3?;(P=_;XgMz0}U+7{fMrU5t7xYz0CHUVN0XRyZQl2FsaGi~`#ACkh( z@Y=BZofhvu{xGKoa#}j{RdlO!e-ho5fEP2CRkbn{)6a48+y8XNb%flq%n*4EODItx z`XoMEosWaVJvF=vG$w5_5zb-`#4|Jl)}Wz^^wwj0bta;{r}VT4$6Lo~H0!vL(ufA$ zit~cEXLhjT9xLgmefwUtSq_-SVn&oI#-bi^zq-tX=gof9{niopTxa6TqD$1>vWZ=L z07`mt2_fQqrLrL@lHjefl`V0fv7`LkDzDsdwdvW;NQ`eb#W=I8RkMiI<1MvYYzq8x z1YG52jdW$RDi!BITV7pYx&7yubsM^Y**OA}uqW2d%h?gGV}{|4shJh2Wcn)nGrc!K z@Uer9Bz9m0QFHc+-GcF=6wMYY09Ut{fBaObF{6iHN!rxaIQH|nI+g}H!Kics5af{f z7r*&tI%BlwaAeNIX=32}*L1Mx)Yh~zjvv9sj2Wpv^*OGB@ZIU$GMD;fD?HMsa=FdP z(U`aaV4b$p^5OWXi|e==ckCWQ*f8s_ZC3M{1Q)Aal;=}{MD97lY5OP%E{=wjI>7l7 z?(HGdsXx@akB%LmrJLa&s`(`LT>d0mo9L<0Lia%$>@&IlzIolLirqI*Z6Gh)^zn_2 zCs(n&L3`{uyH&dg=Lim3liC^+=}IC+l00!ZP=6~k;_wx;iR>qPP4(NYw58j?Aj>;Q z^uC3W96M0|bB4eVG)=qZ^4s9g$nCHlNJSl*{Uyy|=}h23txTypg~slg^xVY`T?MTn z2p@*E@1SAf;2;ZE5Wk@^8lYyL-E?&nuZo71MBD9Fa>jY&U6pgcy9E$P&trvo#vaL{n@ zn5R^WfTotbGxgv{L#Tm@gNcKRgAe7Ji$<4QiRIRR#0Dn+4nR{G%sg$(C~@peR&P%9 zRD{4GniT4ws*HS*7of~l?{lliFl{X|E19fzE8c}#l^M+9+-OluU-DJwL(;2r_sp7OjsrNC zSo`IU(azD1(XJ4K!bmm-io%4=t3e>uF{HR&xzU_wO64>wKNjDmScy3!zNptwC8N!n zw8YWa@a0q?*3e>X{{fc+9UU{@zO){1sQZb3C-ussYPEUL_D650uRJSpy(GJ;b>?d= z*_(gYHLvdn0tQ#|XW$2n%-JSHbz(tmS)L>5`FFkudim;9k+xg^ur36TdMa^wz~an% z3an5C)1`>#xW)cPgzl{$NfEim1E>-%VZsYlp}5dP5wNhOi{Z5(2=#q@DF$7fVx-iG zdMat>5t|UsrtzQI5Swa%h0edXyn);yGY;tmp$+Xt3e!Q+1Smigg}4&<2(~ctY!HWI z=u--B`gS|H|FEa4_Mn_Q)Wy@ODxc#Mbcw{E*#h;__5-*^dgT(NG;S2Doe6&>^y+LD zBwWKFRnx?yNI1NeGrIdpDDDOECiEV&td4;uVHlEUx{Yxbf;6&q&}4w)X%VO-KOmor zKSM(!VOayh%D>9J%e|Fyn{k+NnQ@wNFNY4(<{P-phGm9>ByBoTk;3tPJ3iQr1Aj{y zjLeMI)8;Ee5wX#6@VP^|$EOqUk$^#rC9WkJNX&9R+4o0QFSCk>ukORVe1Nwtnm`*Cn z7e$V&xgf-nkQ{u-4O}XL-f$cB&}6nXrbv2{bD8d~RGExDVoHPCVc3wB`H=Z{hH(vPe$eGB%6KC4Cwp~)Ugy9QAK4~3$P5bGm)MvaSu`f$caeh6au9XWw?frgUPUx-H;#R3^H*ugHA z0N232(kWs~u##uJ-U!B6@2~Bpg^2XXaS5c^UnBW?y!#AsLG+`kB}D96kr$}1Kp|!hw?6shh`k@MtKsVfeo?C5 zW7}$-R`9I)3jwp5>|9YejDf9A=^GZ`z9l;zV-{cWdmqFZ>whzCK}2f|7|=;eX|gb> z9~BkLA>rSfQ)Zz%ULR-0$$b(N?`PAc^f~Vb@GxOoJ_G>=q6c%z|WrQe% zjRKOR$&C&#F7cBRw5)e5c!1Bt)+{gahhshs$D=xp7tg#+_d=#StyNG^XlwPeH6M7L zYj3c|KKBbtR_ws%K(Wfsve_BigOcJIIN^M@ena+>hO+(=8zA?lw&04cAfI(@4998*X%ChI90NwCz)tuf=i2 z1=pxWov=9bn|*gbu_t1>S=}q;VKBtU zchB5Eo-q#y!pDSXjOEtz6S}8s$gG7!ROXR{KkJl9kr{$!O(`qhZ8s|xc8sKHu?hkz zxXUyE-6d`ziJAI!0yq7(voPXNCB$Z~`C}>-C*SnMX^&1!`QU-8yfzbK&gJZjV3{-! zN;T?}Q4Rz!fdlb45M?b1+|>VGZNX*NHS=V;RYhtiU6dV^yJ=Zkx(7?SmFU6IM-|pZ zK-+!)E897XTL8z@@c9=)+e`0$-USdaQ)&tqg$%$4;e(=sqJyV_q=BV@V$Q=!=!YTy z6fD6ySbN0EQtzo%=u_#EAq#UqLGDYDttbqWi}jSH&izYw%Fw;wyXM>HJL5a%yW%_K zyHMLZ;Q%`U8qQS(d#d@vuUF@M<>F{8`<;>|urfNJ!GO&`3|^6YBHgVLA14kNe8Qc? zddF@dS)Yp1Yw2=a&rT&g&$t{dTB_P0c2Ub@E5J=MdNJ5hgsWpI7~H%>jJq&b66-C{ zV|ss8d*0sG1j@-&K-IZ{5a%dYM`aj^n?uRCA*MIEw&A?kI{ho6`yXY5_6`^9864PR>0GvoWK?usT2%9RHJ=a~d0; zJP*Xe1(VbrgQQC4=-rAuRQ?;+Ayo-UaQkAZ0*xfLP2E8?%Useu2EOAs>F&9U=aYAvWZN z3aSw9f=9kGq9XS(%-wBpN}oPki*Z9(L9qP*PaWsdAlWDDGq>3C!f8xc==#MmGun$2 zYU(5f#5us*k~5L=6+9+57PCiY;C})drNdfxEVQ4r2^U8I!vVe%Ugm?yP7tOdA7Zd& zR!zY}`Jk#GtH8*i1V?G+^0-&MM=R(d^;|dh-p^P65gN1Kdbo|f4uZV0W~oR@ODv>L zcJrWy`6cS_k*+J7JKHDCt2B@(sFPFx1dC3%*yy1JG?Vg1C=ppLTVGOTTtw0 zufDZPRb>wKL1zjLNnc?alEY7GD`ADe<-wZ@m`3m}HZbyZcyD;0)#!^6*u&Dh5{F5I zJ=ls;TZUrmbGAy&51t?eOF-C=2^d@mbGC{c@Ln1WAqMXjIJP^2`KX79UUrNHQK^ds zvEa!~>O$Cx%9XP=J6O(@hiL$hFt}q9w(a0?X!1qcr_=raYLODMrN6$Cm5@#Fv7<7^ zek$HlzskCm)qfnuh1cG85|KN#qhHV&+T0!t7mZ^ynKu`RScrm-eg*YdOf%;^+4s^X zg;c#WHWRBIISSaPj*iFYyD(S^XYke^BW5d-ixQ>g!zx({ipdHPA6<+Jpee}Mz%ouf zkN2f}C;CgIkvL0v`&y;)%w<{3QyC|zd)CY;M){tGj7zvjsqN8(+kX#G)Q(E=SlddW z`gXpWnlW--Ay2~=0|C!dstpyMzHUi zG|He#oEGZ1`N| zpV|9GS`715YnC?s_Ek-JJWkq=!c8G7~Uz~sM z?V!HTY}HC24l^!;C04UZ1pSUqpUVMA1QiO;p$Lyli6H#I4iuHdIU`mCEIb~G4tah|H5nt?%1>gQ|2dkhYzryK@)lpP)ZeldoRQ~ zYOdNscl@4eu|&9?3!a|&N8l(%ko%Y`i|^tE|FUQ}BzcJ%F{%s8M||93GjL_?G2N_B zz49B<{YFzpS5Ydkh^VfSl<6^xJYq(j8R|FhV_7(L68%+>{a$@JMy1|@_#87&MTMtz zQcE!i97;?%+Z!hx#uw3FoS0sUb%aP>tZR*Fmsm|mS6Y=K9mMVHTZXo*HJd`qZE*2@ zDte{#EbLxJwpK2kS6zpjHue|DmVw6rmoSa$CbiDnKv=)V+3zvdYyJakS)IL?PW zTNoH}b|e;bY{Wg`-~BQgGsfCHThYcKoFH6b9AVrcoFQD9#?aNyUv~AzWI0}05z#ZN zl~rKLnj*e}iqI`uEgsoV5L|oZWgS(7tAL`yI)`g3lFkkM1>^;69WhGR0-D>)Y&2v4 zfmQ6qe~9m4f?{J0Q^obVhJ{PTOf6oA@g3fqj101g`4MAN5U;)4Q;>)0FybxZH~;lLFby;%(f58o-d`(GQ4F;PGdM zVD&P%zuI>u=*6B@dN)olagTj9`Wi#Jim+Q{{z^Q8CJgO^m4CS_(MI~{#d#G8mG4}Uk1AsaD09MmUV$~f@`y`HTACTslOt0Wl}WN}6V zM7MIuz|FXsFGtT{5@KpFMsnubiBGpnSVG1tLxc(hT@W}7xQF=C1bfnWCKWl<(Y&s})qHbtHtV!kP%MpGN%F!@S9*g7 zQr@Jpz!93uTEMM4@jmc?=Y(ZrxcYiEK8wN^-YFDCWCOCe{MsW#sxu(+X9@B~_C_v^ zNT6z<7!=FXKgLXJ)mMy+ymq2-&OBTK4uS@R2Zjbz<^Odn5vdwHJx6g85fjm67M1~P z_Iz>|Aw zI4(-iOJ-EU&%wdJfAW(33Sc-dE|(Z$I&utqsxt7x)7c6{u0hSot6qyOUg#`+oy_9& z7%4kpaLAERaaGT;XYuZahW24;gs16u|h=o+kJm85r=aV%t->m^fp zV9lof)iL_mYXwI<{Bwysy9<_Hw~m+{u&!}<72d^e>&V92P*l7f@b=8@c}8;HIZ4f` zEnx;cXv?(|;67)6rBSO@!iIE^I6BR*nQ|Ze#`^T(fv$ORk3E+}*4`I0Mn~8XWkk0L zjPD0SRZenB4P;E`jFL)~C9GeIx$V-{UYHtzg?MUB;^&?w9?&FU{KmPBVW{yf{p);= zX--BpMoGQ_4yy)>gI$VDi~eZ@T;;C_iq5f&7Gm)~MFFds0(HHz*cgCOdDW`WMzI|R z9hnzC&>WPDnZ+4g8M)Hy?*0evxHqlJP3u?lA$8kVC6@*2diMl!m+Y5rV+*V%QpNLGz;N`9F%%lp=m4b zG;nT7O52erBLF{qAkDaRWB7^ip7AZpCefXmSoFDsh!LM!k2u(2fdm+aSU?Ju4mQGQ z0p+Vw^Q?$pfl?M~Npf&7V=}$ytDGm-<7<8(8su?mun-^+v~%$e50k|?NQk6+xerK$ z;j@^2He4wh?BjrzFr2OHor~jA61^&Ao<3}>66<5Io{~)gJ&zx0M(XpDr^S@v48Zjp zOreYpd7XnZzLwn3t2VvnQXY7>hk$WOObKeOD)*N#Q zH}*!!X$7k5xi7v|Dsib5zeg1;GrK0oCr%S}c%SAVca@7BB7{l#IpkbSZXm;H`c4~= zN*49%60f8(yqPnX=Ve0_auRHGK@ltsdG!zdU^U829K(X#xL;p!=;Z-2uW48ih9B?? zcqu}0^t5B&XC+$F)Gl4)<-E}!(ehx^rQ#gx++g%tZK`)d`_gX3d!&?qt>8Jna9;Hk zj)3y%IM7Zt7wd)2sExaZ#X`5XwaoI4@>otXFI`Y2Z2$0GJ(uqQo0HaSCgHc~bs=?l zB;dIzIBelv-4|dt-yvWY^4QMIE1sS?0Hcmx&7I&~4O8^1S=`^$3rFEf!bt)6)rwzWT>#YCV$KW>)55xSrB7E`x$loM;WG6dodRG|HX-Q?V5xX@~RMF60 zTG(tN9t(tOTfN~MzVe~=O%{=zyZX<;VMQ`t=i-Ljp>~;KuQevg)lY>2G~0R7%g-F; z5d_7v+j)MI7^G_~i<$?*who5%;1*;}s>T(Y&1uZQt_-sOFaSLOx`PF7@SnZ6 zS4Qtl_c*r0Vo-F>jY6q`)IoEZSu~oM(5EC;Btw?bSVv~=(thdhjnMB(_<2g5$(@rmp}A5#1K zrs-*B1S-nL3dqze;DvWz@{V(>18KVbqu6k~F@5z!a=;M6;9PF2bGJclHu-w&2;RHU537Ef_3~zb5_7GJzbsJCDdO5Cv;IuB5z$N!Y0TpX8_xRMIonik2waJ+H{o znG<*LX#jN|Nhrfy?-;^I+dTFUHY5)?=*=$}$hi7qsUuC22f?3#lipx+{=ssMnFL^} z4G%RGGn*)ZigJZ#fo*7>>K1Zq%G3RXnCX`504_rCANin&0wN>_JEiW0)+7QN5YA!jjUvEh_!Bl#XA77*%qFNKN*iam0zzz{MSn%h$?tnMQi|vjsi)RABYJJb64g6wle3_Gh-Ahj zS-$tv8nZ`!k;K$bZcrAZB7Nu<{<+$`mahypL=6Lic!DyB<{LUCva+Ut ziNzKMM|O0rZX0^Wh26YQU)jYgBOK>=S5wBGx_nJ+p6<{o&hQ8%zJ^)FfK4SifP1ir zKM|*t^?YS{3bE*d=SmkTQHR^JV(Q(c0Uvcx;C~L~)H<76t89=#uVsySYrSWS5{Zk` z3H-CBQItjjVcm}MTY8>pzl89A|85E1N0inV@(gs zw!GC5^3t8#Cv_(Bg@Ucwur*L%iM>otm} z*`s0IK}Wb*b6{h>E8^avkE?c%0`!+ql;DN`JzYy~H!>=@I|s5V@~6cr(W()8+-ZNt zq@a2_4t#O7;FuG=!C&4^j;M96IP_8bGh<3!-rd9(psXCXk=*N_U#z1G)e%r22?ThTT!2anOh>ODNy( zAC@+FVj>ZyOUD_{N)k;nR~Iqj16u_xS3xmRK%O`eV$k3$EVc^*2)}ujA$|;-+(o1G zMO-DZE!sL=x=wyAINCoKt0?i$7@k|bp(`XMChHhc>mvvM5)yZ-ubjcS&JxKKs>4*lAD4YgZHJ_x)@lL>@@tFxEAtf_Bn>2U-Sn?PO| z--^fUypr7rgs|Y&J8`au6oS1%7s$M#-|B80J~6$bu?mm|uyBx&G)#cYz#p6>Oo*;` zD$01KbzdGDEPpS13c{O!A0g+XvbWL?kRD>v+iQAed9^QQk5{(?=5#Kgc`ea(O>~L^ zCB)@WS!j*9r?48h#sov|LMGPuOK$Ai2BOP11Pr8uF^!#T#Mxzunvwhklv`H$W}9L! z(67`IsT=yofKGvNx>F_}WiSc3fe+emleaJ6j`!sFYRDMfTM=)_-zF#72VKWsvVwdmz0>?Z zYuP&)U2%iJW~3?`$2#h3bL&pXz#rogcS@+0Oy(BxUd_L9DW=E4D0{qc6v9MkiTE#HJTCL9J!(7K}$=l6^OiI?UP-bs$?mR*!NO^~T6n(cs z^V~uO0!6P3cvk(qD(XM4ei=C?$nlv0LGX z5lUln9ou`7(N%^g>2o}J6PAJt@yf=Fty&D7+(Kqr2LuPMBqi$}nG?XhSbCVd@aqdf zzq+`|chs*+U)jFe-MKK#p_p(YTQ$p;d<%*4f*=q}ib^;h*y$R&g|)e;Xptl+q+rzg zAuAWi6itP_)I<%2GbJ!FKPO)s&drEgb~+Y55+>W>5c!hy46HSO{Tj=1mN#b$W^*9A98Jn$;ne>W;uS zHp!LaXg`{P2IH&Ir2n6__YRLDJM)E~>grJacB(Ew2qBaMLPWs;6AkMLp0Pb1ukEot zzP1_fuJ_)(zTZ0R`n%uVd+)X+L{3A_NMuA3IS#Z$&Kh!<0TG7GAkvTq_?}L6&Uw!{ z)veim{EYvhy3}2D&U@bQ3p(4&K=_z=XrskgvLt>F;)8tl(QBlMdo9{esCR$nw8q7D zhW4{{Iinu!?`-gGwlnw)q~0L@X7%T$k#9Ctv8JA~9Efc2n#hT$T5e^?R$UP$qM5l6 z*oT4_=4QN-@|4nj`+Ag46; zkn90jm#q~Hw9i~SR##I)yL+^5^D+Q*=1X!Ri z)Dx=Ud^hnN5MvRPCUN}*;iv$kFpHs0*fGg2P%}ZZKZ@?g4SCYBC@HCvkTzinJ9A*U zO{_mg4g}~NLU`)8zUjW@{>t5KzhkG|p)u(~$mT=H^ls202CiA8i?u89%V-&ic!q0U z?PU{P@H0s-6$t`ZpO}b6*%dm(P7RWCvjk1vF+EoO;Q zRSsG?sW%l7_Yyl8qZ3@g6mBnNzp^%~z2eW|`ok6K}3QRkE?)Hk%M^JhI&Go_QI z)g?=J%>{j-A3qfqg(a2x?(w4X?^q?LNFBXguhUoPEA>_SYMn2i`39`JA_NOb`&YM= z*IPTVhVV>kZ1nh(9jT)7s!%vWcD0$eexyR7)UE#Xi3wLoX>modymrR04P>>hx4!n? zv+?BGaZ!G~Xhw{M-L2Fej}E>|N{qiIx%l{C;vE-?4tin;E;cJQD(>V^z%!W%I8UjR z|M2tSbA$qU&Ki?2X!uGZ7(XT2k`M{hqT#qzr8v9(0Qu_&EdRQ)ZvRvAik>EU^>us5 zGTCE&?c_Z*umqFBn14*l|9&0q9qODxX#xx}ma4@s~%%W6mj`%UCI zHTsf7603K#{_{HWd+HRcDo@$I_7~>2{BaVfL#T<^M)krT$(Cl&Et4*K=zXzhpGFP` zZ$Yk(^ z7>gj35_fG^hSv5Y%~+wGlB-^s)ii*V7@Z((y)GPwxrf%-LgSk3Bz5cz?PoYRKnTe# zSV(+wBd^c&^&>C;9aWy@LvWjitNc(NnoH|*h~>+3A+SMjYni0QGM`Z{ocZ7hp9O-l zZgTlw%rF1j$Q>{H1Y*;8Aej&PF&C0_)D}9oy{I>d**%RSXu~LB(*zMZ9qkD@SpJ_! zDJ-*(y4PT%5h31WgU zT~?&YD$~M39X8(YlP$|px!n~qqgpqqT5u&*FdXBmoJ(3XhINETu>NBC>BdfjKS&t0 z69V{9fo5ld5-x-ivmxzuC_Sfc)<4w|IYJI@LB*TUuZ!n@Np&gecA3Dna2h`A*!KHv zXZeHyWFT4yt!{|)k5DhV4-{F~w$C)N083#s8W3KrWI}HWlMwGYQYWGYV;<;F0Rcl? z(|TStoz9!1M5qaFY~Fy6eGag$ZS|eh63%YS%C*$mM-mDdpzU!au#qU}iE&Ka=ulM5 zU*1Jj1;QhuL1MODF#X4KsYuSCRjEYHA(@pL5~-3^iZ7nWbH`Oi(OS5 zCI)rYs?j&Mm!hvr$VmHDY4TLX3n^8vZ#(Yz#3)>0O*SqSK24$Nf(2!#%CS{Pd%VjAC zp^dP=_++e;D#y*GN=&k-rFx)U*gQeQHcNc5x?ztwi4Wx(-Qu{Ma(@hWDKN8Vr!&Qn zMlctqYqJ1wk@Q?f2atF!fWlh*h0yGw#1=#7@ZM14jtwSh~ksT?Gi69v*9zgaiDiQ3D#5TNO~ z(NqaFo=dDc5A3L*P;9u3M;PCME^o#G-zictQP004q&4W9-JiNSUw4T^olw>oHG@lI zr$8f!k4hFPz&Q17(`8P5615Z7<0Kz~XEp93M7J-sV8Cfheio`UBXrC^?)U$A7!rlz z?cR-MhS3y>3K@~jm|!K>dKP;q>r}KM3&0=?*1QPQF}?$xFEF}GNlt3_z>qyu5mg;e z=s@|Vc`Zgvogw3bg~|p>9111vN|gL|f%c<(^4e8U9`0lg2(y@D4cIND9+sZlN8F}= zZFGT5F%(UVsf)k37;Kd<#n_8;z<)6?#m&N+nBP+h?TDT!w$jJ-m2h^4;%q5o@TsP3 zVOi2yRvhnbuC5|!XNPxo8u0NxJI7jZm22f=2apkC9t2i9x9hZi3@4eH37&F~bTt-A z*tMu@-yYm2-Q-<5LL>T-Z_x$X3nbE|hPZ{$dV{u6+oWyQcIb=Aj1>)UUWw;s-v+`U z^vOjZI<+n^=t)Iy@}VB_RA3Cqe1NqE+nVYcg@pfK!}odTiN}s74w;&N68{Ob{o;A@ zB;b}6dsuZ&@&Y(pi#3uX8L5%>f}G*%2z8`t<#{)AlybGgJgdhfRKk+thY=l3ns@x) z=i6qOysgl$BS$Xo9N?#F%~{U_{kR7aeFQjkz6;0Wk9n&!9vrz|6Ws|-WQ(EAPJNfYTi>H2n-Y}FWCA?1KexvV zeay};gjNGS{vDU%@&k?ZQOamVD90lp4PZXx^|ywYxkR0kG>V~exD+z>>PwxquEo*+ zaY{r?v$z-J$JIoWaGjY?bpX#kot$I19U`&wK&}*$c_Xs)V>vLcon7n9wOOYaJt2pr zED}hmcCcM&7-<_i5~m^+yvX^9TKPSoLv)=j(k`iD`kQUp#ep^&9xHCoY#BU?Ts6uf zsQiBWxaseiEB(JD&7V-#E(#CWR$d~LRKA0J!yQp8`N$e(HWyJ5C=V%P$_JP^c(ef~ zYoyIp>Pa8IozVvhH`+GYX1e%82TQoH1ik`jg;g)2c>_y@L_qR=+J5bY|E5PYYw85) zbScGwZ^m<3u_5|u!QKdnfoBmz)XOIO45Fv zYZR8qwAHlJ%GO=GY*ajzBjZr|nFGGq?7@^JAEJ&uN_JN#`G9A!BSiYNP9u{R`Oqrt z!@}e)IlR9D(lwC;VqB!;fd2rI;fZe8;}9Cuk9>{xjMfNhLo$;>{Eyi&={?i8Pis*2 z)s`#FsF$h`F_8$>kI3973%vjQN9y-ukZ5#FNj{A<#p7R%$fRs(w`9@RIt@_@Bn&66 znvS?*!^D%~CGjPfHkk_7p-x5?tHm-#8F`&ZS&mV#N{s6-Cu%C_f2aP0Sp=XJE}Zvb zOiDfk59){XMxA!*$4h~iQbc{JZ0;aN6$xIG&R1tQ{Ei9{S*%U*1ohs7T`Ki+^SD32 zKQd-^oWS5amGKLR0pYtwKL2W5O8DMR0({x2h%cJcIE!2~GxC)L7d3wqA^@)c8@UlD zjK)E7gfnlavh5L3UHk|3A`_~q%o$32=HC%LC63^cgLa43>>ILucw77~$Q^drYAbix zs5^9E_K&up-KMLJeln@e!5ipt=8HAEbHr=c+#a^2NB^h!H@b><(7D4enLoBbe97*k z-Oamn5MRPdQ3&4<20{rh!rUIAQ#K2$5x;5MhEGhZhOi;N�?rViW2w> zl7U0y0;Tr$0E>6jl=I?xQx3CC_&@Rf#_Rn^_n-(`@AV1hptv$)y9}scDUA(O8$sZE zrd{=xj}K7-Rk;wj>fi5<@+_(JnRcK|sw-1Gi=a2ckz7&=iDIq8L|805=FA2E8pm44 zSDr(j4UUbDO%6kw^Y?q3KPe-kCMe_+0?|AJIw zkhhjNXFWdh5m~;E6qb;|{!EkuX99etZ}!AnQvw@5x(GyVqB6)mtS1#5VJjvJs)&8? z`T{ET`!19vu~HKLIovvLbwpps*)d~hfD%ZLb=WV!MYsfC0^Rgtr!O&e!IUP%*l4m4 zT2c6a(oshbJsW)UY;){XLl7~oWB^&*qwEG9Fg9nPC6z88{8xR~e7o$Xc@#vMS<(MX zRE=@D1rbtxoI5nQA3-2kFyCiS89_wNekB)w3;E7Xt{^_Q+c{H}UR2dH5zd;chC{ zOn+Xc`Ivs&iQr&Z;yfF?PaOxTP*~YLG=j(t+9Qi1reDm7h2b^UG78Nw?ZE6o4yIRf z0C=BIG!!p`oKKQRMif$6@>R4i7GlzhJMbJ9z5GfY#Eod@t z8AWZiIiXXdhN+bJ6mvjl0-V%$c+DTl0vIgKK+$O`8H7014c~lZZpd_N$Pe;&kN#Sb zPb)fDByuvG%DGg^R;|8DiYHIxL_MW{D0wE`pu=9rKF59siIQA*nxPxUfey!nHbq!8 zK6SR!v{vE414wf&4|G%8#B6&YrbIUQKJ~Kp5E#diL#A#iDl&A0w9~pcqd|SBP8Rtd zZZF71dhqbilwTDHz_@CZk*myWd&*CkkaBd}VsdW$D)Q1O{a4(XLyhL1kXx>FNoJ#p z@OH05=Mzp5;9TwnOZVXVK;c$BCI43hT0$*HHpne;R7@K{l3MGfi$<5x0tb(3asQ>% zLZ6FkiN#$D39i6gStu(lmLqn@z1wW&N`H*Cd2xRoVK*)K9LPicm+;OZ^_p^BxuGNm zSKFQ6ZcvE((On_)jQ#|q18Y2f=?v+g$H{JA=x6mqjOSuylM=2ZGQ$e?yu*m~c-R*k zQll+ZBvVU5v8QDW4SbGU+4^!_dSkC=orYI7!X~xR>fBm6H~2Tt2veFxCX&)JHp?CD ztGhd?-_sc~B7rtGMLefp)vjrji~}x|rKRBinKJk|=`nm1c^NAL0`xG>cExNEbgEh* zqsP>lETC}DL;@pRFDMr(hBgB}-e*7V#-KdBpx=PbO*#Q$rGl?zs-h0yUFg}MHrTh> zMOeP>ko}eIwynl>M+&b@Zux0ed@Opx%FPC;))YbOwSEyhoeu{WEbU>?EYp%g&w^|v zI+^{3Rp*e_1+P81ZC;tV0>tUMl8^$8R6qg6Buk>+*juC*bIY34H>rFJsoXxb zmH|J!`5!1Po7bV#gsZ2ZW{Jy$Eb!j6M;jh`RMG=$gLhBLojT$gwQPsU&H91S19Avn zOu~uei|6QsOr#-!O?=BB7Y(3ASO?B-`J#S_lQ>6@A_x3i9SGsaO1_!?J{YZP!6`-4 zd$GhVPUUhT@X&$OFEVE>`3JW5^QC^@H$8A!pAonO4}FP=E|RN#3G%P#w{27W!(78% zBV6~L51d~Mq;&8>B5p9;81;ZG=?BqqV5HjrRjfnF33vHVxK6r`*>BoTt7p{66{sqa zj6c|(cL5mDg#Cnv8D^p3XYPTCu~Z@gjC}@&?Nx?ck~h*T&rUV5&f#Agy;E{2 zat{hAp|K&A##e}bz?4I4$q!f?ojh+sgJ3#SOFNZ0DE68_g>X2y>&{= z&yRWZ5^H3XxRpXXu5Cg=2SuLW*q{6V#M)H!Yf95Dd9~jf+Jp4^SxuDQQ%Yu_s0w@! z`o`UUt~*{ys7ob$F%twF8TpzQE54>5@kE{$I+QHSn6p}V%m5OF+G^KzZ5kNvfgvqa zZi2satleD@!@Uf09>ODFq2NhqZKaSQGEz*G&n$haD`c+skOYv2F{7X*W^`0-T1O-l z&O~94u@a8{!~Kti`ctKD;_VL(ENDKjGI*0=`gz|CeYp}_vFIJh1w%(q`-c31Uq)`k zph8@emCIgeC82&w`I&a+|3e{G3!E==GjQJu(}`Zf1;_wm3= zwm@P>Q#~Q~oGH}&$o|;=#QxCp$a6$vZP&PNnIr{pm3u=Y29OlK19(33QM(bXwa$

e0N;0&jGRPR646U&CaL z0f+8iR4GDJL4rXQlhaKIwj%Im4)~j0mK1bL!50LJD}vTh>@zN&c{HbJN9;NNC8-R* zG>0AgAt}$Z2wGEqSJapp+)-3ii3qB{qbVrcP|SiNc7NRp1ZFCSQG zCWYZ&7+oNf)pkDbyWpd`L?}b!sN!rqMA6FZUEo)M)F`sR*Q|dLI2$+@Xo}i!B36${ z>5dtl;Fy9MNgW}r&fj;dEo@7Mv1nO?^|*cBoQpQnUg#L}Hyy7mSMTVnJgX^*Z-j{L z3YnKJt!YsVX%A50uk`QmiCzICQh=tJS>QeBzYF)waDt@TTh<|;WuN%0f%?H_X~4FO zP_~~1-cRAEduPJ-8oz8y`xFF;C;;QGp0v^4vibc^+v!W>0Qy1X<tqv-q{Q;8VFRGZn>yr^z=q_VVSQu|1)D&8R8;t@z9B5G|Wz&-uCnqWzy z^Ce9i?u%sQC!uO>m?Bqqgf#j!Qg{yBXp(oeoyz0EA5)kc%d^aWZ}v|q7^5`n|M&8V ztA1|&ykA7l^*zqL&VA1N1cZS1MG>@KdDeY1Ro zc0tD$zMaitM>|6X%>w*dXJbEt{uM&&iFOf8fj!I$p;Z-(<`5Y;v6l#?qhOMz?e}Se zT-?|rK4YKnB->XyDT`(Ej6e?2FZmxltM9Y_auy`JlQ$m3E@}kRe&C?=KpG)t-SLb1aO4g=Y|HXPezD1z266oDHC&w>)F0LK zLzpFbSY3H`2Qiy^+49P}PfgjEa2Lxf$1N};V5q?I%9)GJ12mi|n_r<)mUCuCnKwe- z$+&C0jHEzo5_Nb=pzUnWEr;mn!_k#m=qy;=vNep>SOFM2Degz2@5FPO*ErTX);ZQY z;>cM{s?P(R_>3fY5GY~F!bUP2&brMMm+(NvN)vrkb+Xke`-O&vDeStO{6FI1S3fh? z)XyU?b&*ey6`@Vs5z-Q;4hEROpo2tY7mTjfcHgc(+wKikDTKI{crk9RbtCr{HcTfe z&3CpDkt>JM8w$kpM%)%d&C}kHmneQMhw3@k&p-Z{V)V7ZC1gUQt^BCz^bxX~E&lOs zhdroPNqUD7rt z>~j)gpGO6h@aZQXv-F0%h7_KDQ@h)%Yko*Q{nw#%oOiZ5N1dxqRi~**&4STk(H%0b z%JWv3Td8cjQ+tdoFK3D@>Fw_FCv$&537qzZy!sS=-{b5Xdv5)f`hCe2a`*d3e>(dE zY6HmoJ$P1?0|6S^E|@^3gYz3bmMfuZs07+h^Gx?V(|N!@nSyQu!^)LR@oxQM8ZEgv4ByzhT+SYq{n3arhf(Hb48_*Y$@>_ z=<-~DtUggsD^Jx~O0`m>jCGE4%H5Z+6n5W~2~il0{2l1>qGf*?VG7f<^8u~d!zG#p z&}yM)gSt`Oq%yNRwit_Y{i!N}ob_l}kqd$K&JE6u&P~qE&iJiX7B`yjxzR#~Ym9Qb z;u2}Qd}wV#W?OLZpQ!vyR_0+ugd{sdjzzs7j}moERsQa~&?6MQpk351X)TI`Sk3IB zU(~}!wPO*qUgZ!0^wI3EGo+uhZ&1sF6+s@}GSt3r(q21-#mEHhy6sS0l<=f``J5*@ zL!$8RaU8H6G_RPUR@GOSo!Ac|Wz%Gr`Omn$lLfwC{PH8}^9Y2Ng>LT}t2}x!fc$#~ z_2qDED}$_)&QnecY>YPFF>`OVkBSs8LEb1gO+aXH3PvMbW>{j;DCQdmcsn@lU)(iq zzQQgej}wD**+QXXrt^BZ)^5BFy+kAIS*|W!Cn^@AtyuJ+KXm)T(WLx@0l%2Z{VGF_RWBzYp(w68Z5 z>~^N|IaC<+gRZmPE&IzTAeTYTvt*xfs@JPcF@#WlY&MzEzMUjtd6 zz6PBb$=f8KMnl&wkh#gf*)M;`Z06GqvTnjH*pTuEdO&tm+tcbRDQg;gQO~t66vt|t zl1`cA<<|V+50?=cn786pZR?KslfB&#ea@Fn04^`S>NbpNw)r5oTCIDxX&FH|Kd0J(9 z`DD@!&>~NEC;Mld>Ye4J^Y4vBf=m7+hY+66=W=&-^?W{SKix2X89Ignlhn!TltdmP z_YXj;x-(=93|f5`M8fBOAuF%d3-V@Yv~+VyKhcsejs`7-g16~DrJOpoeII!w|I^3^ zeVlsUe#}lwFYj+P^kS2Zq}nKlK~+i*$hu*(#NcX`&e2U_N|)+oD2nUlC3^&%xyLI| zL}Ur7wR0{xfJ51PzHj8e_>V1V(PvMt?i=|p9u7K2GO=A6(~e{r;C-NAqU)`oMX6W)^rRP@@c2+i#YS zAluSu6hm49a(3P~IJi(MD3TAs8Jv79mhvcroXB=Vdby}ift)5ffcLrQg@?!#u#dGT zWbfdQXLhBJ9OkIG5!Oi!*%Pt_XFB_#`J3!DXTyHxLzk0%=*c+8otsV6UwR4hBT0w= zPtI;2lyry}!?+#O$YPjg!(y7KpPmbWa%Y8;?p)#HA&&TcT^CEmmzB$N{tNYF3!oK-&j@7gOc9`= z!C}F%{&9Y4Sc@L2B;wm9Ums5%TtA~bWbri#`_NDo@)G2uPPO=1Bl7aAP$H_O#R~*8 zxuZJ67>kSer+J{i@Zw<#Z#XnOILynylac^v+54?_hKBVc$3qtQZh0;6H{FvsnY))C zQ^JG8HqsZgB#k@nwz+t(G@c2Bo(K8#W#=lZ?*PDjtQEurbaKcRW(%FNa4{N3RVh3 z1P(1IMifJv4c?93O?=1v;l+aLIZsvi??< zRBklflMHx6tBEbqYP*pEu04(z!&sZt^-0qX4|%a=kQ3PgmaYdFu#qzLWuN`|yd@3> zQY)G5)GCvn4*#?H;3DxYK!@e%nWbIZwT$eWHu0o|IG&Rzf*5)oA??proV5D$cuG1~>mlfMv0P(?buYBw3ZP$0WxwxtScI>5)KsTpOW|RFf%@aF1J_!|)QlmY^bJ$5;rh#sr0&F-8|Po{X^| zIfMny0Bun4lVF0wDJov57e*arj|P*Pq@&Rh+ER08R#TPjk6K7qXp^dm9B?4Ad*f0L zn;kJH0TUTb!WR3ecB!c=M)ikoQ-jaFEt-#`TObR7`)av&S}@-AK-~ao zkgHPe-P>hq62(+CVjxX6c`sK7*9{;c+RT@l9DG)Yw9Pv~I_+n`Hbs#AwV`|1HXu?lpbwqIN9+pC3~JlL77L~}&~Xl(A%lUBo& zC^d54AcfFsW^h(8LHjd{2sCwLWDcr>37LcPkavU`u%hFFnjkB}f<~wDkjAoXqO!B( zE76)n0;^HhP*#YW5ry> ze)nj~P!?~yS~POC4W5cQgzfJ)$w3(RBNozeAFBNxmHFpFAmQ&xgb&yPI9a-dn5OYV z^MYH1=s}yd2wKkx3hb9K4?az+a!L44a#`N=eaCVrno37VOSr(y4bBTPTpf0T5o1%Y zvahzUu~%udIUkmI$_DTJ;9`%tEZs|v*_sG&rn}K5fkET6t>k>^OmoyS9 zpQBXG^67IOF5YI%wcR#+!vIp=T1h7^y7|7j+uoMZc2Q@9xgWcxZ8yIxHjk7@an?=Ktt$L&mx!6zMikA=U;-re}?1gY?b z^s1pX?@;r_-cTSOdg0H#zw6G&)T)+%CAJQK+k6GD8m~d8g~3I^J+3**Guw093tLDn zS1Z&x%3NihGGAGsEL0XLwK8#SLFIxF8@(Wpe!;RM7Cm!RYu@V{9%vrG)wa}WMk!1( zFr6+J0<-Oi?^{@Pp+8kvbO5Lgnx-)B6Se9i(m3mGl*H7zX-;~gJRId5SWC*tHjz6~ z1~QyvOc`WTdJJMm-|v3Jz8NIa2qki4t$Vnpd%4lHwql2W8r!EVK5iYch>5D)2g@7P3i`Zg&Etp|~Vu-4o= zJk0FIht(u1rgM~~cL$JBQF^?pc2|UyF=X@^^YrGu!O=u27yK)OEE*J_u*f%bhCe*8 zAAt&F0;~#7uuZfzz&7A|5<=t>8aL*HvCJQnx18dYMa|C8Zjg6%a82-m_Y>P#+c+ET zhUcyb*+S(Zg`QQ&4?fg>?+h9IwWO>*&@ODktim=U9h_@}>w<#pCV}^}=vKL_zg8L- zRy)ZkqzGEC4_5mQOg_v-=K8-jP7cb)f~RzlYFJL(cuG5g@A zivesS|`CDG_$NxinoeX;xRDc)597)`Ek?3>jumCMQ%MFK7>f;OK9w*$Qjn_b*@z-FpXr75g7aV+f@6%lSO|QA0yKC z$53VUrqD0Vhx3!j!`b0XiXlc-fWSdI9GvO>#`DZ`N4u-t)9z~zw68U8)|-Xg*>GP& zCRBy^q3uA6n%N7aaXVXz8sna?i22QRK4c=*Xk+GqzE)eOu}K5M>vCRy2irAP4IsZi z1GLZFhuufq0-gLE@J;j68pB*OG#e8GDCWljmA|(X)r?K0I64IH4Pw5*D;uT@bqIPTD%Q0Oxoj|dIi z88S8nqzR<}*}K9qLKAAaFw>`(q4=)rp35`{wVJB%R<^|O9B-*YXmvtgr*aM?=#GlN%WaW_ys1b1FLH z%DURcM3c5Qa�CQi63=#pb@HR)8hEdtZf;g@FXqCF8=b}tX*-#gc zACwTBxk?=`hUZiCmwX5kydm+ce?IAL>Z0zDMX^c5?!oL{DubN*q{u(c_eyeVS1NZa zU;M=UK7X6ogo5yhBDKVz5@}(J$FnFwij`0dn%s;w(|U%g$WdgAoX`{Wusep|VM~dZ zp-9fAXEVoNAKCb|jcQWMnzU>mwE&Fz;3@lQ`x*Om*A*EHyBFjOd&&6b@6|W=BP~5i zBlpGEW-mSbve652BXPit@nRm)h)~(?@@O$)7BESlxtziB$ zr*JAdEoRmqB%G^*QLXmDf>E{mM6D7gW$plLN68D$nUl+HB($~Cwg zy}gi!Qy%Cs6p{sNA{PQ{Fen6NBr$y?mf$O%YVw-Xa;k(@x9zWHRPoDG-VaV}8*Mtc zg?}7bg10=>mqmi{7RNI}pU1!!%N=$21jF>nB53`{_1JaFd)oWl^}@9y7zci0fRL4^ zGOmo0JF-txMy=^bkQ+TAZ=1F=cvHKjeWl&jzS3gDpk|FN2-%dpMD9sqTfgcM=^Ff< zr8(V{>7z-yH!@nv{~}@W&0-+>jGAZu$oJ$R4MdDAg?4mNiOeg}zOd*E>(ht*6IIan zfSiV;DFT}j_lClG>MO7Ag{0E~_Ryt%DGSC%1$Z3bv&PM33VCQbl`h>4sFA{FQ7zQvlX&ddY-KIa~d zwro61(4xZRxgaTo5=di_BBZA{v7~-hwT(o!Z15lP?Y7`s=kfAIT(7zea`>f*yZ*?q zyLVKGE|qx6<_9LMe7QT6d)Y#XQ zfAS{jrR9NcR@om)U^K5l@l&7KEug3MoOlRFxh#p+2GqGhQk~8(h7VGx&S+T*fNN}0 zod^2pz`h_O;D(#=gsVoamo>J zyZrO-Q(9os@@ug&aa!Xa%;)yw_(F=IGDQmsjb`%&Z(JL%o_|qIF#<~DYUEC3#qr+e zpD&7}7}Emsr=JC0a{?RvI!FRgLQ!G+Tt__R6GIA-@~ncpw@5SRI$t2Ic3B6a_SVy2R(N9Q}v8i(Jw^ixlvdH zE60-cd@A+E?U5)nOA{R=!@{=E_>hB6b)v3IjUXnOYX7Hd{6C8gHYDtX@P^!m=q-JY!7`X)s7` z?!Pr^J9)6aNRDt;<4D%{@%JPO#a0+Y_8X)KOk_V4mt6mf8i%2G@N;;3y??MeL`@Zt zDScw79QY-CZ=f1SnEr=v%_fO2K`uRqm*7?__PH8{oKGX!<4cgci^ev0qw;ZMyCIm>T`4Du<-EFBK*ooF(?!{0Hw zLnhXOWD&HL&TH$31}MtBfc5dip^Fdp7U56YG44higjb93upBg?JHHex zKXtxZ-#+&O^{IT3UWLDk7IWW7?lE6m28kvfTa>WlA=FjMWU>|> z5T<_?JRJPON1cxxs#cmOPf{Mt8fnV5hd-dbCn!WF9*t&$Z<9_>n=wYH2PGnL9z-G% zE_iTcHdzdMq{^gwVn>2=0uSAf+>hN)+)v%#xVgGLAA$@0xVXoO_1{2u9^jvhlaG3y zMbPH5I?}}h%TU*xBU0exxzs|Skt;5;*uJclz$Ki2j2$@zX+y8yhmd&rUM|6E5KePB^E4r zl;d=ROg2TuO|gm(LRHmNb}Wr86-w2X(?2#JdsN2Zu4|1(#`02=G7Ah3^0;B#cgJgF zNn!V0Y=XcmqR6Hw2=||?lYqwB@hW(|HkE7cIBjFX{%3}MH(kODC1wPntF*lw?M z`0>MC4@VOBr5_o6q3BAA@OAX)iy;m9_hyMpxq?LSi4-OWVYYXO7C8Vo7dCmj00Flw zpaT78%{CR#!jZVApAA;Hm~Y3Oj)kPvjv#u>&&U-Shhxn{(X{f;p}S<*aaW1E;GPRg zRuCF_l(PPk$Xgi2*RF9LgE=mdj)BWo9txR4`Js^A=m8z#B$-QUtfPlNqMTS!L2%h3 z?M3C}YE#xF+%dB9nu6zp7lIdqmx7ecA%{8c3>oL_8`SyU1>S|;Mc!KPVlTbzcr$!i zk5s%mM4CaI^GU`+&j$5~{Yx+1S(4FEgwj&$wv(r`J^FNR_-^`c`QkQU7#m*%t%((X z9GD+V>o^v9iY>W(VaCSWkDZ0vwpeH2%iu~oWeJ91i$&0CMk=8DBw|kp^_J4Hg_I%9 zr5{r*#kSh)?8Slz;+!!gOwRvIY{1R-mRPNiJkS$Dw|gmS48t`pgPgByw`~MK^-O)P zzEDHw}j%g_j zg{Jx)z%x2XpM-dJYH2cu#~Qg1*sh|y6bW5$fN!*qJIXZXI2rc>;YcYpfX~6u6%*hS zx)NoUedayvC9qQzDe?tFVQpDqXaeCKI#Z=94(?cWx=AI4W?Qgd3;yP}{BSZGypa-Z zN5UXoJL0&uV!Qdt4wXznWv}DZ=5R>E>h=1aIU8gP+rS#ImX!w{WF6WVNrC-*zE*LUut>f?0Kf2 zN;uDY*r5(tNjfy`W1$S;IIP^|V>e4Y5Y(p;SSKc~L^_C8!%hUfjhZCyvmbX0Anr_A z)`5woT#KW7a)ve9F5RF5?HnPfVGsx3;z^;c!+k z?B#&}i0_JTp=Q2dj53X5rq1Ihxpq%jG$kN4^7DJmzwS7A{97X`! z-epD7^Y|RmRa84>Fxh2MerC)|kwr_Q7H2L5hUw$9@fsnY$}1#F2Xk$uBuT0iJ|L18dmNm)Xu;EyT#FL#J{g0?~(y`PNyi(;x1!gGT?;*;C! zo(Yt%-}^qLI+G~C7_Tf7S7of&cxl%cK_$DctZz@Kq6?@5cv1YYGNi&;Hw=@>1HI9g=-racpOG)=23d*5!yb(lzfF9C{U$wjUbgmVw9{YkY)!RmL7j_>celhBrNP2^8W8|zB9%Dvkk31j! z*A>Gz5RJF@BVQs$>EG?$%3^?sW$(&63>i;@z=6NFi_Cu>Zhrk2^ zv84Ew&Q@!0<#)Qg+3BQ#x^9wn5nu((VSSW-J3~zzY3*5rWvE zahM;OrI{7@TwC*tdaC(jWj4l3^Mc2_M-fUOBh@C$hDMKxi~CbKMdH3f9yO{Iq!ln) z!r6Gjf6}O^CA~mz)M#4Wpxz7K50Z%!w`1i(uc$lF88T?17LEmI)Y%czmTN*m63iaV z1N}j8i;94!hG+=#OBydwA zlAcaP26m*I%V8Hd1Z){(M{%-N2FjJ{)^I)57#3hzdGTVNFox>ax1iTKwMjkCAgofU z*1Cj#5_>PQrfnPkB|FrkFSkF?Aj0 z?THXpu?Tp``y{y1fhFYGMshKvJ#}n$f9mG83mU2yfDv|>!kQn-L0~&kX)dXl%z7KA zz^np75wv1_8K?tdH&Kp_j_r2(*xC>sxmn(~~#s#gz==9$J76k>-U^7X!_+a|QEkWSJ1}w4ZfQ4~RgNSao%79FIEj z$T7)$V4Uo&6oeRf+H6wXW1g210cJeA?g(i&kN}36Nwe=O)@K)33PxiCvg0ZMBSAEy zZi#!#MbMh%tR=2P7N{>n@k#%b)b#yPZWCkW5@tOGg{sU6PbSrb%L;qsZ``p%p^8`O z(Kb-Iw*0Z#uMPho5{6y`@$zPjO=J-qiSOY7|4e^qEZMM8fti8WLNnI`3L*4Gz=kB;({y}^sD~;?h0d$Mu%_u zt|gE@$SYGAV=-QN5WK!wF$> z7nJKEQ+_#Qmp2dpaopkUuTunAsnpEfzGKX+cZ@=s+=;|MlC%^;D@q1f8q>u*cT}3c z1|>Y@RXPbg9VlNv%%s7;7xnA)kJe-)euh9*0KwUIthJ||P#!kdP)VHh86cbNVrfGb zjmAnLG8014{6}C9AV5iD@MC}IHrT+@Y)R-jsBw@E&LPH7V}XbJ z3a>#4PnDLwv_;18iTi=Pw2shrk8`hcpYvMafD-b>h3_|F} zAbJY%Nf!E3l!PtQJP|g}&_l@vZ!CwqX}e|n&2J5DjH_}ksRN=tGZ|iqN25g^*>?L5 z!MGH)P!QfE@hniXCb!}k+dTZc=7asdRo@B8P4$Aj9U?$E`UB{>zZ{b`PC?DfT^aJ$ zkr9W^r>8710;LibKZMIoBqJYo4Rm8iU~ zAFfrvzRL{IMj8YxhR$#iRbb4pNwa3Gf`Bm^{n$S&=&+1D1o*Q z{g3=YKr+IZc7gVF{&U{rZ~_(w`0UT=9(F000@ihb_FHTgmpf~B=TVTZWOw%so1IJX z-SQ!5K0vclb1DZ%y=-1o5~(CtIB=^MaT22qIAPhty|Pvq2`MN{i)B_z0>!%RDxHm* ztB9==isckQ_Ah6Fcb8^i1NRc-i=8i!*aM;~bcZa0&J%wEQ4xO$@62(`b3{rmz+By~ka^km zeGBaVV6|d`dABxk<;y(^$3()0KV2*twA#dXL!-=bT3EG~H0Ne0LLnp1nWkwZjrB$1 zHl4;|aU`v_DvuJ)mgL6j^GgR6Rx!4c) zsWQmswgLWCKiayVNUX(T6tDS&y)XYO{CvLSKjwQW+nDIc`)}~`L263`om%vls?7r^ z7Hd|1i&&Ef`f>XSJ1YaaTivm602$Qgar7yzQrWtFJDr~wf-xk(D?rXrtLiIkq%to6 zJ+?dOn|-Q0+7LFOT%n#(2~d2M3#M#)XljRr3&a<>Htt+ZLeOiGnv~H?64VMe9LILu za8P4Xndzj5f@Z*?o1tQcYkmABZRC5_ntz7+3tQImN!?wu;I9NLNY-`jXJe zWUuJ#jyEZRC`owrUO6;Xp#+#@H)9o;3c-j(7JRd!v-OKBeDcbZ0GIA?D11N zHT#xpCoU3T{l!%ETx;O)-7D`2|1a7RDJ{^_@!+PSh!h&b%dr#EMrMP?J*%lELAfdoMVir+Z%?>c7pV;)VscWzWcrhzOQ{>0u8Z6 zq_7viwOr1`=rUA}o3q_>TC-cnQ6ZlcU#f;I^1xFf!E1OTcp$Vr|A6;|#(0 z1KsWy@D@(0zTF&A`{JYWRn>Me+Hgw{KSjUo0S z_JrI8j$M#MSQshDl|lyFDI0H0Fa~RJ*K;%r$%8;lK!I`!tkf%4G*KKUQ%b^Ax5fb3 z_KX`e>T~21!Rf1srj{~n$2g*0G*+Qisjr$+(HB1Zf6ZO_FXBsf*O@76=-1*)9*p^0 z^OE<(m#~&l!b`>tCIi}TGHvVGPfTx@P@YWNdReWh&K@ol%5ftqlTe}j_VJ%kfKcHj zgV&o#(6A=?HVA>01O7dZUtNclcC-J8+O<(Z4Pr2ifV>^R^O=tV10dMGRG9!TD3dT` z3bJKMc2WtX$3*xpz(u$OOz7j3f+S=hSDmeLT0c@C$Z@O~qTd!A13guqf9E>>gk#2Dr)wx8#z$>N<4K|tBv8E_d zAqEK@{EQmrm2)ekY_q^>fn%)I56yidSnXT4oFk8M{N(q~gv2|8sZ& z^a+aEo(x!1A^^}GvKM+bsC%Mi&m;R|`xE<9`!{w*{)HXorD^+p(YYxC>_!o^j%g8Kuol8(r^RjG@tezM6+{zyd|24g zh_R>ctL5J5M%-EkBO&u4csIapJ3d(MR`;kAydvio?BgGYO*RQoW2jS<2l_a!5ssQJ zWsoz+lT3qel&hLB`RtK*w3AG9U4J=ILj(cFM7?)uk{nLCklA_x7&DAV{_99G6$xHv zgZC?SrZLN?Hli}coV;!x1SZ;96KIa*l@Gzl)*j&=>6UXT`$AETG21wz9951f$CVSx zN#&GsS~;U|^dTJJ+k$hBG08L6z!>@xx#uQ6i91==L#(*8bb9?S_v_A(uI76Q{gZhH zje|{u4;20~!5u>wN1x!}kiV%steP96H12OJI!m+$+nVYcicThzJV2>#5?L6neCzK#IQHKqh`vu@MT``u>q8~o8IgjTc&w>gnS-MsHWm&5u9l}D>s=zcJGRx{k7 zBRv=N?P{bN9Z6NAEC|)I_GSy$B4{(pFT{kg$5tU23HgI2?|JWXB^tjtBxosQ@aD#5>2!dP(C?C@owM#wuHgqudc5cK$1405hX%cTcGmojnj9KzSFx_mpYpF zoFWqtShE9Dia1)*(g9jTyaTY$;6bgl2p2i09psp1mWBrO1gbTCcgh*SBp>3^FK z?tig%kg}W%8Tv^p)-lR~No#&Wby_k3mKYOk6K#`hlSSu0BUpom1RcQr3saoY{naVu zlcNYII#D<%e)}JVA$_Hb-HyLH9D`f7lge{>7LN@rGE6j^R z>OV!$y1^kh;qhB9mU+23m7Nn!9okT-(Gl9l&MaO#AGMEqr|iqkZfRlv0DCfbhK#GK zR9gU-<+*%~hpD!C0-sr2^%p^_Di|%GXXS*3geZDe2k`6x>Q5n#aA&rkAM10z2#j@@ zb|dt7$4XQOaWgIZMkwre0Pixx(w~B#bFphR16+F?zlx1e=6|%O_PmS42TOUME8b9* zx$hj78hd*u3?fs7feQ98XI0t}t}FURaRrIU{smwprX^Fngr|T^n+m{quFdoCsW&sT zYpY%IYQHtyLr>bnw|4g=52FBrrvIxv5avRl$;Gw{o$gIv5(z3-^5}ptG+E#srl0p+ z@Wo3s++xZAZMiYg)⪼VCB{d6S76bQ!82OFYf%z{POQxUOE0cIh)={BE!MjLtAP? zDMP0Hl;S{JUTgdO=pU#lj@6=Exoaa?lp7VxUv_F2>G}WE^2&K@h_}IaEw5bFOsE?# zSzbBf8UbT>vJ$>jt0oLqX=4_v-#_c|1ac&fS1d8E&cN0KZ&H9LnLUp4A8Q)>5ny16 zg+}LuOGgO|;rCWbn#%1{W)&O$jrI3bwy*t#dF_uQCCC-a5@gCN&4bKZpvjgTaQq-| z%)kNUdZ{Hp0&le1ZL z$g!i8B-9L34m>>nKGh5@fK~%O{@pDHB|8y~0PF@91GL5t*=;KPhvwQOO%M!OCB;au>4>HpOw>9;9NO>0%d3qw z##&>YvEJBVY&1>>&jiV2uVsAxGRS%47@>|-tt^9(XhNSD!!t&gY0132Gh}QsW;!+- z)s9b%nI0_siD2OcU~K1&A$Z3*;lPWN^A<1N-Djqg#BmyK3htR?8(Vq1OXAT-Iaz&# zN_V`;4s0<@ojY!)#}%U(=X+8Yqg7!kIOej?zdM1PDA`6A$T;FX>KzK7x&``M+^jhV z{0luB)F$V7=LP4%VBD{k9wMiMBcw#b^OWD$Oup!4&>4~(2g{QeLF<^B^i2mxW31$p6 zdfIBkecEpg_qzM8!%If|h_vmjC9GH($M&Sn;vTbGIo@}7tW}mP} zsa4tMf3l5YWOau0b0E>Zr)DM|v*HYxL;tm;NS^eS|$(5h4- zS8NgGIU<0#U%I!T$qZ{vit$oP>H^x0x2lJPsj4>7)7t+`+0uBRge*J@SDGb`ifIE# zIwlPu|NZ%rWv@|ug6}|=9j=z5BW`Cj`J+R8VMz23MBwVuTu^88WK(>0luFsC0H`(syU z*f+k|)s+-Aa=Bcb+T?-WU<~szYZ<=-E9pxFOYaD2v+Y8aIII$!v~$EGQDligtL9zf z3B1H_t%oX6pU(rG0agDWp+lx<#E8u3WF2Ilc^`I&#z~lW+{t2kkDpPyL>lFu^H5oL zzp6dy#YA%k0p!9p!yh)woN#n}kZ=hl)(D(FE05B*?SA-Y^A&W8JStxE%(XzNy>0%q z_pApu$g4!0TGv*rC&DG`ZDMg}R-2;~GKT6i9kU!lXK18q)gEA3V0j>3D|eZnnJ+ra5gM@ZXd#Nh21HHiw^_5hy(Auh1RkMz{}_aik6$xP*)xVnc> zBMGD9#3Hd6<0?Q=n?>)AYEy1J{I=zl=VrZXUfJ2|O54!t&gSDwi{$izgwV>;6SYg9 z9qnk|PNI@y3pE`k#JPn^vh$5CsEeWXc4L&zc}!>(AA|^6CuB3S!MnrQX>5kX1sKzj z55Z5>cv(xn9m>ITz<*esK5DbB(9W??MDKhctc&De1WuCE=14J7$IS$ zcdt9_e5O8E4=Ig`xs2KKr~r)J28YVWE4V~zL$7`9m|At7vU1A=_hUWLS#_VNRl)?H zJRtHD)~g%TJ;q*xZg?pn;@THeGFTD3q+@w8PHsGXpCL27WU0!WB~v0A^X7VV%ELsehrL)lV850u-&#T^UNBOf(p^#6*(7?x^ks8xo4706o?p z-$E4=ofw5+Bq~(uJ!zG;(KQW}9CHEaD;y&v;;r0=6GX5N|SZQ;pTZZ2th5uizdIajQON+>*zp=y* z)j#+L^U9A|1&GA$_)69SV>5LSL4^G|i{n6XbxV{4|1XP=;y6piN7;vKAkVS9B&HBmh+ zOt@n+uSPp++g5vls)b&I(nJjW!4G%3T9^=Lf%l>7uZ$}P1Axp1-!GV%D{h_@LaS2- zPtwN8buY*}Z6xo)qh?@nG#!f_SykE~?cWA#QH5CT(;qhuZKAr>aZMP?7J)u1%bn$u z2ap3O*2^1Z8*Lk7`@}ZZHqJH~rodE~2ArB}f9U$ucgAQ*Hx$nxDgR}&UxtPwCD8U4 zV?3XD#(Fq2bbKo3LST<$i;I-;Vq)Mnofe zatc&GsnnR`(CM}1xfEfqJ-GMVe)&;sY>4xv9Pr1uWH)VzmPmO{$fQ<>Y(`JWr6lC# zKn12bo&&xwjKy{;Ib3qg)pCtk>`j;hh2AN$UcS)61pdFNuPN{ zynKWNcnvz8HJ*D}nT*-W9A&ODPnoYQP!=kSlv+h{0OWyw&S)~&Yo_(QYC^4=6%kYE zfR+kVsViizS2w8pT>D)IT*I}~_A~a+?O)i>+9h`^;-LDMs%oRO1IpBC>nQKL7SX7fMM_s&_O^N&2~pd$c3S%x3q zju1)Qqvpcfw0aFB+GeUpg;4ub7Pyq|eqH@{l0C@Jk=g8FBEyk166q8RSyUUAY51-OlI~ zgXu7RfW$ghpDo>>Q2Asw_*OeF88zO2o9&I2vZ(vOa3&hxtY6)v()FvA>XOp9;6w9I z{mAHUtuNr7{l|5mTxqmXq=7=Q$ux!>bV{IA7b{iX!x~#ppS{0t%$kQf#V&H*1$Op1oQ?+@Ev zY3{2g#b^iGRbwZ_9VfBH;yfGXmLmXm5wyM=e4$l4LT>igqXAAfc<1T!bpjGO`1jT0 zn&bv7?2yoP7;cXH+eabgvZu^SLiPYyb%fJs{_d=YpUGK|z_&19=JZM!ARB zlNopo4C&!e$VT@RA3DeYCwep@kN*;A8y|)62@3C-d)b#5++mf9Rugs;T{zVWP(HBp zgXT>I1S3)?=4`poE?uBddbfF?AGU8$XX%tPCw|vGAUno{k%MyinnmB?taGh!S(vPw z{`%{Ae|@AenV3&UB~(od3z+~$A1Jz(njj$R*yKWhc2J9VUs0CIuM)9J7$Y)((Y4pCnl6YdJZ zxNgLm(2PSg+i!mrI^HmD8XLWvyj{ChTFcA0D?|SEp(ceIIHci$LtJc|Fa3$XZW5TU z^q8tt_FOwd$gFQAl8T!yAw2{MMdyP-Lr@dpl>MHCU{HC)hq6{e5wyN#T+)*Y$7O-{ zE5m|n7;X1)luvwcF^+;PAj5HT9|21GWiiBa=gc8OItv;l765;QSn3nirb&fU_nD`T zOkScS0;&6^tL-<-M<^7Q^}EpHjORAjk50lFL{U!;utAbIap&vf>IoHbUol<#JJ98k zqeV;$vbQbHRVtxM$Gt zi>i>PuH3X*%87i}Ro*(_6{2xC=?U3aI93nh<(xd`su)(@mNZr@T-uQ{U@2sT&syxxH>K$ReQ%#}?hHCY{f$k5CuZ@SsBV(|g z&1UGpyU0nW3rIt!7v#@VkuYBD?Jyrh;-M`dc%AtE`8lS8UD#o~k}|E#eZNYn+1lhF zpKSq~4gc9xSYjJz*sr{(X-huT8w!TFr>IlaDB#sXZ;*oywLWM+Wanvuh(sh{GYFJ% z6PMr=k%jvg3L1Cr2FIeSqn?tG-Nh(_tmF1uFeo6+dbOMO6Q6vlQI*9jDYqUq<>H~T z$ba#OoYRue+M2%je_Qzq_oqcjP})PDp5UcT{<}?Ggd+w>NS*n8h6Xv%}ZD zR7c=dDBkI31WIGOOzNSh2lT`d6F_&nW9WprYX~vs2k^b+XdT2+a(+G zVU$3;1o`(pRP>d63a6EQpio%S^Gg;@Voj%jBlfvIoZGOdfZkAW*CiDGw=N9|AUKz! zO>)C$F}V`q+cNE9lgj&(C%fsR*FMmlpM?m-iIL|{V$ z`-R4sPbg$Ja=`zo*HVI@gipOM)^BJ1%wB_1$=MCpg=l{>ze?Uob-TC79sSefE-|p$@ zu5Ms}VRB%G3__q_3)zj={oZ@48=v2K=gE5x)jic!_kQm;e8OcH+mA+Lhdz+E z+_A!8E3koHnKWPh&j)88TLK5h_hO51VFrs8Mn3Ze5KO>-&w~+`*gABFPB(+syfRUJ zPH!WUICHkJ09-uZ&jo))xCSU~+q~!q0!DJDgnc9OV{hm(*U5syFbh5EXe8~$L9`H3 zIrnQ+x4wVfz0Y~fpAMf|Z6$5mJiS0DamB9X5)1i!(KhN9Lzr61 zBr>3rD-QyD9Be{@4$%^bu#i#G;uS0pPvOrJX27=1;D)J$-D!fM<5+)6HTq6m2(>!i-8y;pNXiO1= zSpI~tyWXAK&UWm0PJYjIclNuKrc#oAe8w%%u)(po1R~YpW!iF#)x=LM1>FqII&ARj z(A@ZiLNQ!lh0>v6s|rdbmBISB6hX*pB#|>&J+YqhLeG!=L|Q+JT!`2L%E=Em8LkTOdJSB z3_?M=hiW2CCu<@MYOpgACYs9Y>L3)VWdnnZw~#_nF8IF+ee9!r7LZ2Y%TWASTj)-h zF4`F{B8sO8xvdCXata`~Xw*_Fwf#i)iE=YKoXM9;Rn!qn#v)b%=X3OMob_8;ughN@FGCF@>h#QKyEO`*93)|Ygc8zz8M|rr8 zu~wC5j3I*^RWFB*-0vrvSZv#|%uafn-8dXC++2>RZnoJnEW}hgdqu}cU&zBa8@|BxR#SmVte!^#1NP@K-Iub-Y))e5!=DaBzVpWPp z!o0wS%;gW|tBLrx)GaaC;&K~oD(gMgMmdibUHY-O)TvVujeG%vtK29|P z{}O?1leKF$Y*Jh!iPWv*R8sGO*)AaXht4%>pTsdgy2cBsSl zd#_*LONz(6_C@K*hhGs%_Br-N>E_D^$@d;a-^iLyzRV*A8y9U`Lm|G=uU zv)}4$Sw4y&9)GUq(}t;H6&ic#{?AwUw7#yRomH7@?C_scL*O@{djd;iHmXw%|FN8& zw4(ffwU=Aoq$K*&GtNRn)y&YF*}PbSdSKe)2AsHch4#dVHbIq1^1SeDxtk3*N?Ya} zy~m({vU0a&bZ+EBz$6@*#o;lv<}=cc%F=tJ5{`TUiUp@?D)LQ533CTfk9$SBQN1CT z?oA3+npu=jF8EJn(7*N49~z15PJKMfNtkkRQi&+b=?ys_S-$haAseQ<2^is~Iq_4+ z+!_Mzj-_uMR%)v>(vzk{S5Z%;J9K{6ruB#V$9CakcI{6osV%!J&uk^~EU(yJdFSzK z)Bw@$%CYnNTi;nEwvI9P9-`@(R@s@skbVw?g0XA^(x&!C>1N_g9_109$q4lz-EpdK z^rL^Xo`P5J4jp-`O)Kla>Dqp-%1LIklqVFkX0RbvK4gx?1rJKNlbPBc;644?Kd=5S zHMo>Y^`fF*iXCD~oFdR~2nO0Hgm#7YkKLbnMXg!nlq1inFpMkdJ;sPh=SZ^Ca$AF1 zb{iEya6tg8fT3wXH|RJ9ZaEs1@BiSx?aV}@CA^EpvSx;HoM=iMkCRgNx%RtohXol+ z**qRp0aL1p7C9V)2=XBCRAtJ&w7a_R9iFnDq(l;u!-G4Y0NVI~FLn!N95QXWacAgP-R>erB=4WrH-!n?2%@1f~p4iYiip zM4Mfsjbh-FO^UcgVx>#gUAssKwiIy9Ioc*xw?h>07KoBW$EkPFMuxSa9v;5h8DLa+ zbU4`^LS2NU`Z{eYLlS{w<;}H?lsZHyL_W|7RXjWe_*!oOT z@kCqNclhLuZgT8&hfek3C2Esrkw;`=jHF3qJq0`&E`~79!bE`iVhFFu5KN@6zctKK z0ceiZOzqaFn>?F6xQDG>dplYaRTyxGP`t!DeUx&QC}-6x;sMfZWyu|Dp}wvEHT;-H z0VHN=-?$&SW#ju47Ra2jcT4b=idH3uiIEoqUplwU9ZYtMO~t_K&{=)yPaGz+o9at* z%3(620XCu}kG5?pvvhF@lo*U-{a13Q`@F_z>rO#`NumW}k8+WoXMfy&|A>{&8sD)2 z`=WC;&HF&T<9rBC_U-Xj>a#T}x-$h(c|4<|?tozItD=fg6;+sn;HeuU8O-O5FykFT z9q*suU+3mw=S83se?;lDtr#LVyd3{Nsm*uEx`ocSRPXb6uFPY9pW&TAL6i`p#bZ0DS`Mwz$;_iOke?}5HW z{S(6g-`l(p6!8Gi#PKC4xa&&^^}v(iZgxWleBc&={=mg&BQ)p(WUD4hW7zOr zn&|sn)~;w*wQJgSO?F{dxzZHshUY;1R&5sj!8lI$8OMZkuzPiXy3Sfmk@9veQn|@VSIF~ z)z*!Ub(#`b$K4adlfs-J2znnqA)9g$6EM)+j~4!n((e{r}cUtACkuSVNxpT%EJs6$rt*k5j z#)!eBvr!INB=n@9Tqa$~l3Y>@F7fVf`fn5+It3I3srIMJX;HVc3;37#KK98@i@${L z@-;`j_M(ts1Xav~z(nuGOcut#OcqV}j8kIy~yL_Rat(PGFtpJIuo_p?LS$#eOZG zUkJxyWJbh*@%lhcgU6OW!;!{&LiT32ozf3_3#Aa55*Fq67yqo0Xm=k_>>Jytib~|w zOcT}G?5K&82Z07wDgZaHiXe0(!zvreP?~Lu#C54>yLf85=j^-!s$rFbTqp3M-rXYS zw(_WbPR1qPAEy)3diQ;&P_MHbvc3VC?E-32*iC|~zN#cm$aq6TWC+6fU6o2=G$$h= z%|+QJWmieHCUm7Z>sssF;os@s<=^e!;}N`H5UZMRR$}RSbs)X;9FcF{gq|z47rSem81{zTXo`#|}BrD$fIdEPPm~0PnI-y{)s247$B3o%f*NYwKjG+`9d$6}GzT zy&%U_s%Z?fEn=}{NiGCd2ieI6dqHn3r8z_x%lDD(I)&z9XLUmj zIS&jawzDlQh6Glfb-LmIorThIWFV#X(*Lmj{onO20JFlg!*jxkW_^v3-DX6c6tBl- zRHN+(Y@9+G1U~(BDB~4{r9HRNnHx53_b@ZzclAL&%*!y%_;TTY9{oBB&oI4dwa zFei{A!C}AtE$FxKPnZ*0ry!*=lbU6Y={F4uxYrl*&QqvB@yw#aRZ%~PWO%k&nd87G<<{a1>)8P7 zfavG(K&!pLd!-_`ILTF$`!!nGbEBg7NrRn%&YA?3g&M$F?-nmi>n97!a@_Fh5BUV{ zBK0BQD%GxubVTG;i_}lbjnSIuhjrJa)#UqrGrEZMl}fkJ75CR=K(_$k4o zRYKJQau%dygz|z8;?o>i`^(BBR?DDNQn|Ap^|O9VfnEZ7 zYp5&wI;zr;5=yMeFH=hDiN<+ z?XEUtNh4*S(ZMv|#=3Jt`n5L1)nZgSXxxs_kM~9sVXZWo`<~%8#h)26_q8$DLvCXQ z5HuBJJ`OJp$HhBv7cvL@b(z=wR-xiK3dT_DJ5+NRF1D&+=N~;GdrRilOmd*3!7pwe z7J=RzzURH~ec*lQ{l@#q``G)$%l%w)s1$c%=Eg)QVsyd32Dfmf`iyGbb<#6WEiQss z3k48-?#EFWb=tbHUTiq!N}@&2dxa+noA^gaXaES^M~yGKG|$J z;NR@u;q2we%_jp#jjU1=OfRH|d=m`*voz9SZXe6jKc z*Aa!h|E|Xw-F%yPm>h_=M+JC-c$7b*Y(7YQK|ho)M8}gatokAvwz=-~5kf93j_nf8 zXB6#{6d5Ft~of^=6*wDOdjRr5$6Y@S1W(d+{=$S`wy>{MWVDpAwB9NISr#H)9; za%ctNS~n_o5f=fT`GRFYZ`Ku&5_?jRnV*UDFA?pLnc)WNUUi19H`| z_9GG;beo^auMd4MG;=I7_K+#Jc^0bjux23D<1;ya2`HUNh7RG$<5A-MFWp!KM0I?= z;eS$1_ymlJ#2s>7{l-%31_H6?e8P*Lvod{=wN|3pv4}S=Fx@@F-RPd_p2fX6Z{5Xw zN|wJnbeiUtU)^cbfjnvl*uE!a?RcxH9PN2tco1)k>XFRT%H;MmBG;XH-jYAwHP}e}d`B^kssZ7w%eX`;sg?jVy zoq>N(Z=OtUGMXiIhfa_E@^zB=xb2^;t39^1P`GZlzqX;d7o~PCWq<99+(XSX63M%yxYjn>ve&v79wed`Umhg__eGly(ts}V_40j=jQsF{Mq^K9BO zlRKX~m+8CbEf?W;`$L<}tv z3%zNZ9aAoozDhqm8(?{uo}VIjwum{mUJ+iTF+$HY8Cvc$hkEFVuEu`%yRp-YM0ZvS?TMZ|JF{bf4c+Y~L7vR}pImZNc~J7|HQHo1 z4+GOGBgVK!Y7E>HvIVIvbu=Mmi;`r+56?7?IVQA{x7=I)e(;_agE|{`owTai=wc;J z7siXQXw=C;6iKUBDj!kY;xB?Q1)Cx~M79v?o7#i=7K5 zZB*#id1kAJJylc~bNc(#6go|0IO@swO-G{+;TNPGx?i6+){y+|4ZSbqtPHOTugF}P zxhivY<|mn}!^ictG}i^F|O9c(d`5Gh^i;+wxWs*ITu&NvhqhK^7cgoMdY<-U^4dcclB7JmL z(W(%FQ~ajtkSUIb<#fLfFP(Ls^CZx?FJMCBS=7$3&T+#T`(u_ckA)F?;<#2vh^TK{ zow9n|C?lsi-y@Yrp7)+d-=n5Txg1jsfiiL-qauUeuIj*=!|$`ouCyZ_r4%Sow378` zxFM~)gw`GuMC*xIja!C8LAiOUOpW=+48agKN}jmv!$c3~`hhi+ zdO|j-K?OI2Q~CnPj6F5RJSlT^^+E>)hwKb+IAdOBl5dI>4DulGm209j*lKK#Bw0|q6C#h}yc%iI<{o-S-qDT&(G zplogU^UMvIr?s?%U$7(4<-lm|<5X@CCS@qu&Jk)@vf-%*bQ%(p6N#$~e=|{ej->|P z<6i4GBIUBzp2e{LFnz(qZiSJIBw$CQ@wjb7@b zy;G$So^Nk{w8LY4B2&)=|GMzSj7u3yGve4YZh3VA?X&QB#{|bj-gPMHbURPwePdBD z%LV_zOgo?G9PqDku63>tf1X7AA@G=eJm+2>CCNP3RAvb!=t^O+fA$0AHb7?l{3O&8bka+6 zi~qarY}@M;gv*|z$ryquYc2{V} zVCD!rcXyFdV^E4p1p+;OjFY#oq^~J0`>plb{{?NhSg*fA(1}`T+*3`EYsX!A+| z=pUI(g5N>9lsap#Ctte z+0OKZxmGk6N?GTWK!oM)nk}-d z-7R?(k|7aiu!f4jfS?>Y%RFU0(qG7&JEBc)66oA= zxfAxL@=4s^;Y?R9l!_uyTX+ocA|+H_Er*Pjga4wU?8N)&3`mDZ?P1i zqTh-gmjWsT9?12EoVz}IopXdolJ0t>JC8hpSkXOy!vYK68`Z=W6w+7i4q0oQYn`8~ z8`Q)=V!LC9V`t(>^$u-xMu!p}xFTxb1BsW#5Mw)$huQ8-qZRj~#f#5t)^f{? zWth9eEuo_!GSXl&2TF^Up}OHC?j3lGmn0{y-Y=jO%C6uv0Uk*Aw-7>!*$QpCfKSl^ zo-d#s|53D#WIj&43;5Riskeg6&Unc{Jg8)&(2Mah(J94f)Jd4L-Juix8c-_ z0Vi5*(yMJli9LwJu0g`mD*Hm-R?i5hE!qZ4Aa?U!ofvxTKCYFmYsE%Z*9>zwkN z<~k?<6oEd@&pByg``;Y!eGVJoA54J8>Jno#IXT^|ys;Io-(PaHQlSp2o{%kI-7sW- zfjuupNw+q0Xjbida;*9EevrQ>JX0B?j#bC04eEGxf;v&1q)t}VDXKbERi~+%mm~3g z2YRp6zV^J(G##TBiy=%qpj$V{SttY<+nj{67P}9Vrv|0EWSttw*R(I>W1;=Ye%fhX z(6#9*=klPTbrI-fqC<`v-r5NHnNioGom>dP%g&J=HhRqYUMuNuw8a4@u@hl$cwd-W zhU_%6e_8=fn;tO67?kwJi)m{eaxR*&k;-SJ!X0(WnxjL%W39q(r6oT2&eGOFWRjQz z{%?X*;fvAt1%p#&fHhm(gi?!6U( zejr>I7#Xcp&!X*%jr#Qo@Fka`!tvO0mVX)h_>Vx$A>ghTkz%d`ln+<3zrebl;P0( zK$p!9TeRwu(7{_P-=g5QIpA*zFU^p81#r$I?D53g7e3RTYmIKh%O5FBiY6fv%;{hZ zHz;_|04TO6;#fkpNUXCTsE%NQ-c9qTK4*hy5w40_s35=?eqA3Ib>5mhTTJDV;VLisfG<=T$ezEpm6EykghChpKfCd%{%p$ znXBQAJ^0q}US*&1+-bNf2nNu*xW&;*;%4vIg~G8T2kT2Ay>)UEL0|Qe>$YNYNSS;Z z^y9iMRSE-alq4IYlv%Srwrpafuj?;E@kibsCe+ntl*Y^ZCPs?q;YmQsvgs-B@5+RXDIbUt)S8xmzFY-it(2aZrJf%YD;G`btgW--+=Cw zt}53MSG8-XtHzZu7P3KRduxrN`iiKFYTO5Bxn?7r@CQ?OSwH1RVy9}!GdJZUIb{s& zJ!kuJP@CQ!gTQ;Q#qL(7Op30yo4KhK69$|N~x3zx2yBYz4^q-Y`DVqgV-~Ug_^zu1y|B=uSCP;0tjAmU3Rg+ zEWT^>!H0x$v)Xy>`78$s!9u^M@l6&&$mpqe0`KpBZ?=RG`#2lCAAD%FiDzwoNru~O<{sGlq5MHuxtd){Pd&hng zH#AW;jTl1)zx4|H^a&jH!v4M#8bthxl(vrtSQnMzHFVmwf7VtfS-kOowp^LDrP{Qn z?QZ%7gl_vk@*VSSr9t*6Y_kjLh+}ipd4)+7Ho2K!i`Xvc78=@Z*u0BUs?O&jQCMQ4 zSWFDXf|giJ-gD*cHmkg7RKy)Xgmx%}_UFQPGp1%v%e)AeV3Cyk)h!|F*x(nOBig7g zts@($oc0wcU69$Tq*YdM(D+m%l6R}tedDDhjlK(g#I;;vxbZ+q5Xw>bLLJEdP=xvr zNm3qmeqgm~I@3Vts=>z4s6zL-g9*9kTOKlK5?Cw+6RVyYM`>r2BbSE#t+ni5i5=vd z1LHExVbnQSi;KD*uiyd*Jk4MkA#)tpzoZx`@xy+<`+)nP`;Z%@MOcxj6xyE;A9NcT zTmprt%4LvsA$(DMDd)tRbxC%qpo1rs8}NK^Wl<&Fa4X>>kpq zMlAX7)>Yjk)*p_k)BfH1gJt?lP$*tqz3dj(YfvWAyAZb_a1l}jVTvR{5;=B!*8{Rm zUy;?mPkaef7B!PaJYl{_LImGD2wV;9@z%RPa*uJx5%HMjl?xe{!!%A`+c!yhXtm3J zYPMi4qe&(UWk*C(j1uO^93dG2sY=7iU$Outap7zzNB1r! zY0K++J_K$!o;sg7_bX<7W;496GuE<30t@dN08 z&7EM6y_keL9wwJcMD(=dP`x@zVZ&a5lDW=#&SU-;_Z-UsxDwv#T^w2x`Z%O(j{1$I zvHzcZqyZsWZlbbrWFDOt3(@IFeEtBV%<@?v&``9)rIyAO8hOZq=wnKO5XmYIi zqJ$&Oh_KK<-mrWbQ0y~1jYQ96r$M?PU{FvZP5+;CwFsEF>3S3_AUZFF{Ha1z81ceDH-<*np)Sx zbW@966a^5x5ym*Rim1|R1)xECNATPXe;J<#@5*&TQRdF><1>i8a|!$FdOgufi+zUU9+Y&XX1Q1+F_qZ0SNbF4`!!v-aYQ_pUeANWg(@s$sOw(xHCZqO24;|>;ayLSK5OQc=9T#*GigkUcO{bul* z_nEs%TO^(T=RwARgl^q&H~}YtH?7LUMY8u;lM*E-(qM6XLHbX{5H^`XmvjnCjrxlP z0;~syP?tj%t=_=0+K_%jHo$F0G>jSNc4NJOSx2seo~OUFh2b7-|PzQcB)r1uVrF881JzrcQY9~WyL}dmlplo?COOs3Mn8;M7%4m%_b$1 z9 zeIYMVp+`a9Bu9BgBKb}DX)y4`wZCpA(1un;?+EJG;ZH)bx3u=MtCdV;W#+P<*X$#P zh4USHo}eH4qm6BYNQGDc0Tcdc?hTIJo;VOkIYv9`9SNK3i7*M~`R4lsu2Uk|&ep1h zB#I(>WVHEcZs8%dh9~}KEmwGAbD>UqLFF0S9 zO?|XPC~Bn*b!_=;CQ#RPl_2M}hf3R`(_ZRInl9!&*5rIhERo}_+_?@3`BYmUs+dW3 zh);~HKb;Y+8_Zxl9=JK0Blcy&UoKr!+79Q1xz|F}bKrz| zF1Y8dyF)+wzK|cQVUD|(`)$N>G5!&K>rnUVJrxeR-_#Ld1z7$fQ&$#)o**}~zh#eW z%Cjg%5|v4bcYHqjcPz?{P3Ik7)ODo@1qomXbA0WG#B;9y!GY>VYV(%M2mIbD!_!78 z+Pv6Cmq`EOLcWh^oXSZOATV#p@MDe>2A7F_ zcq?MQb~eE15Cyzv5#z~J_W+1aJtWIJ5K6A3nMY%Q+t(lQ&c@FsRho^-s ze|_8|5~tsfM%*Vdz@I95U&wtBzU7$jU^!q*9nB82s?`?gL)H+`YP|FFFpT=Z0%YJQ zWwgS1hH@ox91!s-Bx#a{^5%eAZ}=9 zE^Tts*`NS2r2b_4WNWSUx1s95-KIu`I`i_u{|()7^d+Kve88Q2DP>aCDy=G5Ifop? z{#af2D4fv68fs)5sAzOrr{K4s+vD&A=R{|E+AE%IJ)5n_(X8k_AiFkbr;p7Js&XM? zv-Y|7mYZsfU~*vGhLf}F@IBv@Pu0)epyQLU4TXu(xDzTxqoZm_-o^!%Nv+{cc2JWp zIOl+CCwTfd4XakD;dGj`fNicrOCNhEENd3A74V$?*9BIS=tDgZGFpMLkY3WFen!&b ztc3h5&UDdn{n`tNLvUopG^rGG+-EP&@KQg@j+BDzb<*nqqt|Z-X;eEkZ21oLViHS= zOlM_oSA`Jz+<^^0@KWju?HE%KIWN6G6qtBJJ3J>=BS=rleIkrGJ#-+Yn3pDL=M(q& zmMHzA?4+~QS8mk5NtLMu5PS^Cf(z^0bp~yxTUJ_2bGkpS+Ozp3>-D~>cZW`EoNJx) z{PX<_{MSQAGTGic`kTqL@{C(FfkbI#9{8Wqan-wjlG0igrTXpNZJ-<45T&JVn1f38 z@*wb3WvHqcL0AHjZ4S}M2Klk4Eycb=TXAT(hs-w1Ad4MVjn#GpOubolmhobe1rOD_ zN$05&@!Qb7(nolpFN~8a0^RJ5KMPw>1`MVUwR$_&@1hh903uHBqQE!$b zT;nl}I4ea7<)1dmtZ^=!w8Kf|d1X@nXpT+v`U}~#V2nT^gl4%oJ$Hmg$%o9j4$=h? zX-MaQ|B>=ou@Y0yT)oX|rp1Q=-0RY&Dx)uuhHhh=k@`H`0IAf>@XWas!m%)5L@FZ8 zF(?3}>9Gr!!E&&P6U7pI5#P}f*a(g(XtVO31P^zsiZDE$seyV@Da8&ReU^)t(i*F5 zuGpe5DlL)mE&;7go2MJ23<}nrm~gQ5Gt;GE)gwy8s>YlU2ip%l{ZFkkTb<+_qiv5c z&RpocjZ^?}cL%G5bc2qGiTom^+J8tptg&4z%)=K-nS9hn;K=c&E6<7W(WKJ#yO(q|0lQ zEMQ5yQi$9QP~7f!WH|RMP^+}WSPbDP`a%A1C*8Lc=5(mrEI}M26HM7vtl_m7m#iFL zNeZ2=`pZy!KlC8l(k^_ zfgaGQ(Kpj4ySz5)Z5`6;vQ}SzD7fuQ{a2b>0z)cV9kc-8kcjkht-|n+uwu8S_ko-m z)&36*F@jM-?2JQn5=>1R9KW5c3a-Z#xS2sMqG+SAGtPCLXrhK-I9b05A+QCs>#PJ_o*>QuWQ zxF5aPojE8{$)jpRv<8C=;PS*bGt~Xj=)J$UZYH8<)1sOm`L6FB)tAa?(vnM^>o98a zNdH$l*T8ZKHa^!FDQHW7_=mbpl?n^)td495ZSpdsDU2BF3weL|_X)~GWs)*knW9Wp zrttv=I>9eI(#8Bby#4F^y^2y3|zo+J5C8968uB>1D>1=|DnW2cUyP`a8yz)e$NtlxB z=0>mIBG(O(URN>s>IO2SXqJ$NE-DM^pZ*1PV&sE=lbimgQ9t-%!nWCj4dfL+wd>B^ z(`Pp;6#r8A_ZpoO9#vYt7Dkcy%a%DuEdFK<8R`7Bo(nKELN1u#yIOt!l7k`y7NGlv zrhUrTQS;~ctZO`a9{6{$WLTIsU+p-foM_yU;5aG&q4+AA#)p6 zZ$#ZLaf51B(>~Im+NZE~d#td5`}uGps;1ft{e+IxeUIAFC1SW0*F6`4&yhu$7;iA{ z&@qxjERy}&3gbQz&Aab3ZCxe8;VhM=cS^%f+%pj0>DV!&2r#+EAr*bK|L!vAuv6Wo z?pApiJC;ec(Uf=1Gu9KS<&PD6nPB9&Y;(pcE9U;Qb>;8PF&JfNKs{xi2KB(z=EfH5 zNRQLr!$_=k$zU`{Oh$tTsh9Yt8P8Ylc%3R~lI6*2$8Z&uC&=)q6xtn)m?~Za+;$Ug>(XXz*+yrqNSEZbT9s!^8^Lzu#qt@HYe)o)sMkyb zd)V%w&+d%AbqAPVJLYkE_kVy@unkuy<^i#(zRRZNMpYJ^NP8f z2!AU>Zd8d*%`N~VNSj7m|NbySNj#vKycA0Bou2j?N@aZ>i6bVK@PgA0ONdbPIp7}| zp{sTTNi7gWPk}$cs7RfkW9hP;pgcw(1TrdTPEg|iNj@}REJ7R#rkN!&MR1|1mSKv# zCh|WP{3d-Rh#)7NbD~Z~zcefuI8VbI&as~J6E)YObxz&YhAe4hC!PB}qa#y&yF-k! z0GDSKfnFb3A_yA{ptC;|RjIG-g4D?#vuoOEA4Rn448G4H#;Kz-tUnZtiKNCu@gq6m zaf-|_auNzAEnc$+ zxo&E%AfK)wp@|RSH#;;luFx}y7yYmNFQO?%Z^+*6`E#4=FSwqMYw%DfMiJ;$PJaI9 z*}myV){});IVhMCdYs3(;IRwJFhrxhM;)4 zN>CMxPLVNwX)K!43`g-YUHzNIU(C)RkvMh)r2NpKzg~2tiv7@)lpp%)SBrLE@RJXj z1dc!ZbNOfHy}NMjZ1yu-Q+;Op-#1sx2{50zoa!@+e&1ZTh5AeZWYQ?l_~z;H`PID& z)5H?{;cCkxo94Ar)a4@3?+QpX2z+Yq#E8WG--MpyJ)DG^t+1JZW->rPj6m}kssGEA zaaE+FH-NC+)0jBog0dTQ+N>E`g_fYhqO%=I|B00e`cEI-M!%a5+VBkRx4QTIMP=yH zjeb((c9o*9a7FQRcr$_dI^Z-BU-jCHV#RS1W;@_wdI?3QTL;@mLNW;)1AIg0dne{w zDz9-U*D4>ui(51WHn>av7{ZEl&@TF84|UiBW$5~i8RVgsLc1&eagm0|_(&@K7#a2! z$M00|(y!h!W?ETG)GZ|S4}Wcq1spjLx8SlRf2ny*(LAc1WWc<}d z4r=9)H6gOePqa_QIT{?}9h+b?aIsy~s|`MFxJ9&P-?mvtn1gCJYB=@S;^#yQQ@U)U z$$%(N#5t0vPF72A9(|L##s9Kfv_3_Cgr21YMe=}b>?__+PVags)j8yqeR}!Y2DHgk?@^<0vzR3`& zB;ajW7VoMMJ|@|PtCW^eH)twUF8C)!7P&~}cIgj|q@!PNkpd|o1f^4Y=S)uuf>|Kc z)n*wvB@mesHdNly0Bz<(zlUlJk(g7HUE5|UTN%Mv=`=Yq#!sukGf$%kbfOnW-q%>K zQ8S3dxFk)3dH;;2A1gyICj~ne8x)l3XZq_BGq5NXtMFd<0#9#r76P4E8ZH&dW`O5Q~_nj2bDi^Vvv`VeA zkyMBAV@QS*;?#>nOM!DWUVYgyv9gtPK)QhMN~SzgO0%e;%CeT-1hqIIwwaJpEZOXc zqE!Ns`Kmx90(bTQ27b&#*S~Q;a?28#6bTKVrhJt-!Hr`qQg=67^~s}G*s6CtJ<^pL z7fCgST9k*MMLpCKOh)D{jxXJOn-~qhii93$SQVrPbfVQG@B{!0taXP@Ycqy>m#81B zA}JT7k&p-^q&<%^8%$tMbvaVe>~5boA*tQ&DZcz)rtcXIfsBgs{sv&5G#A+0XDs&vNXzol9-JY5BfThe2WCSiag{JWyXQg5I$t}6850Jq#`oObBLnR-GH~kSc zpZmzpp=ZbmHHh=aKZ{x@J&4}9I7?gWoBG*~heH?Wqbuhx}7C`X1Vj!$03TsGKi81HIu78@lS8v zTSB7m%jPo@nzVQ(_iMB=h(<-v=e&+M*%s_}bPOsc8k7=_$gs(=`oTX^#jWTP*VyP^ zSud--_*-^vwPa1Ap>##YkTGfpP^U%`3{h4KrW7Jm{6e8GZ{3krYMfCF;W5fs<+wYJ zNhd8UmvXK+qHH#K>pyFVeJn28B?Oij=NqS0MWqX;kp(?k@+f{AL)&HK(Xc?j5 zsOwmYI>Ih>mFn)RO7nIi7~Wn?`$txUHsD^zM~k{qWG)FF>bUa7m)o(7~k2Ux177AcIJBbAsS6B%Cy*pHWSY zbcbFH5n4~5)GlmRox{1)uyB8dyHMpOr_q#fw8q-b?%2AV7QqKMI@fwb?)1omOj{8V zHs};WXq&6TQ|V!LE-K8s;|jC`L-*ETQoRx@pU7AttSK)8 zBx9A%YxSj#n_Q&D(I<`b5W!iCAxyi15{MDY0w^@yV_7jCR+XU?+O5c-z@HRYnw`D2 zPd3xno>C4}M#+e~$W&ov6AQ#3$TA_e9B|MCq9z(bd~$zD8GmpvQxFS8UvczHF(@7~ zXE52eT>yUfdt!`-?ZwNaG6SCf{czE28qZk_k?}4On3Bzgiy^!>_&C5na2EG&+o&9G z{1w%b%L2Ed#DQPa`WfR$dFD2|fXF0SDyf?gA)p$q)^9*ltwbY?h#$y0%k8zsh&D6A z?bkxk7x;Ls92UnS?L$>6TgKjc*?Isk$0EwT&+FB?S;PWxnP_lCu4HMdW~gt44G4 zulNm1yTowoUBEXtBA-eK=V)lwRtD|OG7e{{xXD{8tJJY2hZrmQY4kRk*4%dJJ)!eq z{}I2`WeX$6`#|nd&%jq+WhQDEa?BnHlFadrGUn@hI)y>8qmN-f$*$1unMO&7$ZyzY zVg{$C9wn;o|DorC|C#T(&qA+r!b$z0z|_YX?W^~Fg7 z7%`YW79@4_4{MLNMd~;SCJz^ zt$X~PDD{1oedSEgZAzrsGT1;XJqP@o{agH&lagnj*BCvRMEHJ?bHfqUY+ewo`_%by z>yj#k2FB=v-+P~HuU%7G_FG@wKemUC1wV2Af)^h`(b%?hcb&NVGi#X!69YrB#ho%Y zT0!i;E1Y(eN@Z#EA&I#U9k$=Qw~}Dljg|J-t{HWow2SX6w%5+NzlBizw%A{LtzkO( z-f6aUQKg}33Ax`T4)$6$Mk$+SP9fhr*&f4j>jW8S{KhVbJL)&?BM_QU>~qE6P{ zlwJJk?j|o5iCrIZ_m&~3N?tmB@;lzqp}~u z025n|V7jUGNjRgf?oAPyt+3$p$cI{(5Aw`FVHZ$a@{8jH7tm%Y0;DC_RRFygrY3Cv6l;h!x;y4+HjVAKxNSG5C8s!W;H&@!n)yT zns(GpZB42A4Fau|KxC^%SBa0+C+aeV+cjMPj#I{-GgqiawdEl6Q&ag(>;@g0RlA;q zjkq%R1~(<#jbQF2xH?GQ5{}^gC8IVpj$YEPVhYK`y?Q^$I}}XCU>jn%j7p1wH7Je% z72MHfeqcJg@p&7XA6uyWJ(HsTV^XP%0ltyrU^a28a>mQf&;qZ3&MdU-~S3z_Mf~uF`q}6<=9( zB61+3)?eo*%t$2kn*;u35&9LM2w$~M%>!|b7Haxr=N5y;PWcctH5x2f2@2WWq2qbq z1s`j-Y&FckaqrwrHcMw|;^3`}IvUB4a!~Bh*$FtiK#mH95E{!(6|m}FQUtk$cVy0{yM39 zyT!g~c1VOe#fBwf6>!w-EaR&aGf3ek8b<=gZ1RraSsWJctsK({%lvW#E9(mFS4LJv zR!36S7+nB9i9B;1*9>=sTi3RGW}Lg=ByX<=bmEA>Hkp8Bm^l^%hM7nOejt=o2Zjb} z0*5`cH5*;7S;vlMkgmMTOsI(d7471`{$~ejTPbLMBXaHRZqlmdh2~4 zZ-I}70*WeZwx?MFky$7%8r#ALk-4#A+m>| zgIoBRn@Ve~MWj0r?=^0dgfQfrPz0gGtl{g>-H;W*@4F{gpQEBg0-e88p(eWI2P5=j z5$HSJpL?#uau+=SL;Se3yt2_CzAJS2G?JF2ybKJ2lmQDMxGwTpWIZZ{nH!akdHz>h z{Sxaih`tniM#BJ27T=MK-LnDaX$SSCip3PGx4=DvO*wB7EBTqW`xwC#c1+WgY=U?A zu)961x1P2q*-6jX$JhK6%`Z(Sy+&{IlI5A~=h88&Tj9Mt@i6_lKF|d#N}-QD2rRnV z_!336NG#P0F5yx|DMT0WF_tz+6iv|XGy_Mwxobv}^IO$>KsM|2buQyZkUAO>+NE6X zn|!_B)(cIqQ-q3Q(07Jz2XMNH+qhYvt_CxiW3+9)2Ypv+UwbUcia*Wzwg~jkBP;ad z&Jzek#_X1EkVWW_mxI$XlSsZeXaNtaY5$_#MMjN5i8B>~PGn(aprOv7ZHUMoi0wi^ zNx=7^9}V;8iQ?jo#kG!w-CncWL}tWsLC8a3R;?(r?gUyiLSiTLDh5LF*O_-S?`7W4 zvx{VQYT+3=__Jh0x1Z|=w#o%ctl+PxTH2ZCy1mdZC@*puf?=(XR;uA?n#oqM=@gnOj>fg(}>$9u@uNeFc(jaocaFRtxnYjzod(nnCk1d<+AUzcf`o})>@sAUnAr?O1{I$MxZkaon z++FYBbn^ASSNdMw))x?5`a;W(K)-y25rF*lQijD=NrT_&;qgoivQ9@8*shigqR00q zK`sO~v5j!f&qGo`{cc5cJ+Yg92}^jG--f&%Y~P^k{07Ua#t6mEbw*B~oaC7LVxeQZZ<7C? zM3yKC?LsD%jb}$Fqs#^WWA{Gae&1U6r|xxbT^nDvaYiF$S6Kx52nSC~2tA$sk5p%) z2=q-F-~FmkD<%#mIn#Hb*SoS2y?idK8dcU@c2F`!ALS9I=!i&E4Ei`f4;VhHtQ|*= zm$(ZDy9;T&A(Lhef?iu951rpQA30AdkDW7=MkSTM1tQ@Tflhn$nhD`N$H)zXNs3bf zk=20&H1f8K-SD`%EAF|Ys`LlY|FoA=4NFJ21Cy1z0NQN?^tEg zc)Fy#Ba1cAQp0TCJ~Q?vQ91SL#WA zOf?ft0L$ZGH>S@Shf~6mZA!z)uP-Z9pR-UXEQTyldO&Bh$u4=C%x{q=fT&$0JT$OE zK@sRXc`0$;8rY@>$l@iqc9{Bf!rRoZ<=bIw-BS)7kPs3Ll4YB%eIY-=ZfJAR>;}SL zjz4ae;!q4>n!9O@Q4k)6^Fd)-R*0!(MlPWl(PXDV1V_Lgd^trKwaU7S{&7$=0?u-h z|0OcZBJ{&;1V+(#ya+-gR4%-YaqPK!ULnYgguvTRZ+iZs)h*yzx*ne-z#y-c77L#c z!l?Cyyfx0X&d=2iDm^+zgKKOD)d497lDRcT_4b79-TEWIqge6*lvGL~3cN0WxGEg{ z^qbRU$M=&E`?jxZAU%k2ikt_5I7s;dFC1hgXD1Tz;2{x-XosmE$dcmJiQ2JE4J*}PCl-lq!)h;ETMX&{bO3d>{|Nl&e+foP#4%*b z%cIY2%OKDx3Ab~S`$WZ;7P7Zqi9PS5E?bXjZ=K;b>I^q|-tiDIAviI3&z}&Nt<-00 z(gU~{A|{h}MR<{P&R7N=t^~ww2yFI|1HO?eC6tYQnFL5saV~rGA)?-o&_Eelnk|{%Aw;u-+kXVu1Bs1zK6cU8a?Pma-~+o0~JCn+^1c_ zr`NNsXR~#jH7ojep>IN`lF-4^QP0K=9&t28nk276$<{O<-T0_u)}^@jKr#TuA6-J< z5I;J!-}?UEcvD5H^1fNe)>S}fPt^fb8c7V%EV}yNDel3437@B9PnGL zqC`8By~$qCCCSW^-Nf4LMr)fTT$tuM_u1Yy+L(PI@2q0_u*yDzGNLs83Np@3;}eK> zk_`|GUCH-@!(k5iZzRdCiT8(&%_7iuM{cR7T&G=MxX!q&-`?A>?kZumBQ#%NXTTDR zcE`j+f3&WO@5zLTU=5))YH?am!NP`Ngya(M z-W>BB_q4c@t6m$LG3&rgCpqu)A+s(xRx>D34eahzA}+xfI>udD3zQ64Ho)PG9U%$g zs}Mr5wSbx!h%eBIN=>j9fTRn;YQy&fv>%B-I~|jit~o z>K2(!s_)D^@g7yT2ufe`r=qWX3;KxqwJC6@v%GUBeFD@U>`dctc=%wwvw>IXs*95K6+5kR%&KXpX1-Fk(aF|N1rUfm5K0co`(%xybz;;_axnoF zT-@#JDtisu=)l3+a}I*Cv26e$3^P>;ZVl3vAX^+u*?W_eD<2z#0Mk!@^##Ngu}ErH zS$|C%r&OLJ1GsqhBRDttqBO})v2Zu+mMb@1;CamiZF|os(S2-2b4BA2OA@5=Dzy6!9 zE&K0lsv0RGX(|hz&5kLT4NB~lYdIqs@G|Keu2!xe+2f!lGbIqFCO@RTbMn=SnXL}; zl1m}-X{I1?;LHy@1I*Ln_K1R~{sL#mm))zj5pNNQty(esQ>z+e)J7==MmM{Ssn2Ms zSEzg~sf={oecJzp|BQcJW<%!q%>9vxnOu2Q2JJ^FqZN@_2YvpH(&jmj`Q<1m)cdjbc-sCiedMfWoxT&u}$>sDK zTrl_6t8-5++@LU#dz0Hz>*7#ovOSKvUrEB8cI2huGtaq6#v|{wvpu>}2^y2z$z7p+ zl}8MbK(mr0Abg|9$sFrM3SpCySwlHw(fYtF#Se*4*g>_heMu+K4n{UQH#s-+$|8jj zYzZ6<91HC55&jxR#7V_jR+L5wBd*bm$C9X%v17&Ov@B^~$Z7D{cjYHXRb!=*L3YvW zP_~>anWej<)_Z-o>I)`QcPP1Zq|G zk1TUczey6Amtz}#rMgNL@Gf~%3EEQG8o02`k5ai;A1fhN1o|c4W#1LwRo^w=bI(4o zJUQt<`YkBGr{C8f=nwT8*DzNq!>~#tSWh;HrvD0l$ooMNo{9OtX}UqNeaWPh&^*uI zq4A^Ydp6ZL5`um5oGHs6P}u@u#GpYgJhl6Uhm$ z2^|1RzZ$*MNg+st1347hUxzZjEhvp~HE`nUdCEtk0D@bbrp$>6dPJSe*P(1v`hQ`W z_wa#D)L)RaSZ6xstg7}>Z@Ume4?NtFfpWvH(BZG@eINP8#MxD>qeR*`0A>23-qu6T zPC-?eF^bqHcC?ew)9#JAM)aZHwf|bzt=sQXPz0N=^=zx1OWds9vb}@*=Z}%qGeFM) zI2~x29rfxCI#nn zC&Ra(TrfqIF300~Wm7AK0^=<|)LXZ{pu=R+(k_aK237-L<{zaipSq0 zk(S8O$gv0?f4Qzt1Mk;^%@p$?P~k)AyYoWTd5U6_&3J=gKq3-jLohgAZU`w)^@MB^ z3I9@{aE>7!f_k>)DLT<%6Ca@z};}p3izOB>q-(z+lhI+ctkEQSGXN7iGsV8saqBW zC~(FAC^{ZFAv((*lt1~W@A>YxqJ(0;3Xs}0AF8mp7vtk^Y5k-yLYD0&dS z<(Ti_pB%fy*ZL}FlUE&x#RrzlfPBR;Jg*2s*Y!0H&NhS9?I;3$rFtS#~uuvEn7M`GQzT_IImx72?j*}IkwPaivklLnv{dr7r3cI z+)fk!+KE=8Q_yO!pTB1d-$DgJyv_^e^|fj-&1_+=7+>m z^6xPbkSz$EjaQ4QW1%PSGRt0%)J*J1^b5zVB2QwaJ>kA)|3?Ju&i8ncI6yW)Op-ShyNIotIv2rnQc^NMio!z@+2e&g#;SB-H{m=vxIFuxFCS8UgmWAP))^*kE1RXLcse=#}g z|J>H!^7K?|%4@t~d*vO2KnDNG_R2laer4V0U*Z7(du@32jXXv@!mi0N;@T?|c(YjI z?uqnDCF`yI_UkL@IjYV2a7Y^!%{H1${axsT`i*I{;yi1!cUT!1p4NRnIaj=FIDOlpB9x=u{b zFJ7D=Bt%zezs9-NDQakAV91(Qs>166u*dPsUztQN zJXbD-b~jy&ORNO`Z5p*+aDL`asHro(Tf9{|Xs09E7ZH(gpq%zy?Hu1yNg0E523Y6& z%$JPrsLS3x(P;P?O*FN>X_nHmah5{y^GYC+=p{eTI1{mK?jVYBTNB zGuwW8hVI_&p6&n6zk7FQX5X3J_ck)Cih?RkVO0p1_@b~vAV46Ifbd4R!bk!VBfKeL zga85Wq>Fp+cW*={J>9+U%o)v`!>!0L*L>z@-dBXG863p3nw-__)I?ppS_o{kiRwm= z+@~ho975$j)neQr*N!t0*FmkYY6zs-Wm#0(tHll~4Z1wK(UA0MP3E@-J(e8f1vd^I zAw8cGe+WFAY#E)JWJ>>wJvT4YG746yJ3EJ~gvBuqZaF+j^GlvhD{R&UZLCsR46clF zesU3^3Pm62G0kQQ??P?|CRNXZsk^>L5d{M3nSft=v(pxBj43oiXQnFD5^^}tEGnn6 zgIXwD>FN2V6&=!2!vXcY$iF2CP{GuI{SmaKNQCUR?19B)2L8gVORI^jIv#k ze)_YNMm2WA9tPD|KW)D%CJM1Jnm{XbcSW=%O37~2K;c~VhV7>9mJP|qA=uvwP<<{h zH~R|2)-|UOaT{=usZo=ZaoTSA^LoRR>JO$+IyQ#yn@X_`f~cpyOWri|U^!Jp?US}E zN)kH@0u&q|d7QDel1j**#TiSh*IjFx381Jt@j$nHYrJc{Q(Rnlg+0Xt;v=?E7vCdd zt>csI>9c{Ao;_xLgCD`GJX#N9^_uNDAobXJ)Eyl4YU-0ANLMkyBK49d%{JwFa{^e6 z2>{=sX*SKa_T94-tfn0D)>7cl~4r!%b9Kt)44PX3>!fjjFF!X3|Z=qF~`#|^efy@?+szaRX zPK8Llu3*U=qDrQ_{k@^bZpR+Sg}^6)iviA;ck;KojQH;liM)l8PzlS8(Ges(trU7^ zVoAYD@MlU7XFW|6VVns}g&UK6`^^i)dizbuA|T#h8J?+A%CM#2z3ktt(qTB8a)dHx z*7l4RyW?+f#D7WYx0Nt{2I@g+Y6B6gi%R}-?h3r1onXv)sblMTx@KYAi2*^ucM?LX z@w^H_DR|z4&9Kw8%a#6Cx22=|jIWTEwI>gAqioNScMC}grWnbfqoT6lG3LrMj&7GJ z>T4N4CL$l%MY_nZC7vU3mqZ?ajI!orvyFOR+=#1r+xB0|r+x-~l^xR431bxsnVr4@ zxrvF+S&`3F>+i;W+bSryCX$mwq;fRO5cvz9iHE4KQ_g*1b*IwVuv4LCK^Zs3`H#(8 ziDFSy;va3lwtC%JH~ae7i87`CqEk!$j(R<5IeTJ2@K0yX&zGosEx`s0C?es$(1h-t zXC`#Fd=hnoG=KU$iY!~g&c?ISoPEs-^PIX5eBExoG6h$;BcHjx%qALRaUJe~>!nGr zU(9m9@5zT|UZfW(cfVVFpDfz#XGgHVb3o}>IMZph!rpd0J@B@RngQtZzAv4*xJYK^ z82fp%?7Xv;Pd|9m&VE|<4z9^74=h%wD^x)KY+FX-&+scnr6O*Id?K$t-({m-DSD7Y z>WQZ-ywr_Uihjm9aLwIzTGfun2qLVvg%EYw(_R0O$~e=8+-w(Trni;wT;)W}@V;VV z`li-9Cw0Xo`V1GzJsy|tcg>~LrOUy$LP^A?amQK5ImdzgMwtt7D~0;|uD3qlPGZgP z_y<67ip5FlH7K@1HLwtP7OAan(y`DYy1a{YM_xgB*wTgt!_ zvE6E*;FNTlu`Wy+g6$eD#hQ$-*CkmS0w7FWH3Y5%M0&jlC(|B!M9%7 zpv>_x4M-Ls-w%3T4VYGhwFcdNT{kKs-4loVwE&CEAE)<>*2f%1wXhdxCp0qw3=0Ab z${7bh5nU(Yn#7zuLASIs=sS~?@P6HuLmKzrK07pnfLM~YrC3gn7xQA%h)T%6Xvfr3 zjxJa&_Ig+%z5ZY^U7bsMYrg9cAc&`YD^^dsQAu3TBl{)|=|(4em)I(}j|F&yOXiCK zwyD5y_qx+~!Vm6eH^Sn=_hL=F2xks(4LtfYwjC`fion5xM6 zC|M4x_otg_+=*BP{!3nC#TYp z@;)g_KK^U|Iy2clZQVubAb|ixlghy)s^$Ul$9l(%b2_yvEO&nB;>A3X>wGl?=Gg>5 z1uVf?0c-rueQ@NSMCBLelHfZZPbV0Pfob*xxug;OBfNGR zR73Ho$d3+_zgn=>~YR@-gUvX*w$`a zVq+(wK&-=DfCIJFyMevI)#&=)oKxsCmZO(@!n32)u2uxvPY4j|!7Hm;;uH|d+&3RP zO^UaN&zk2F=z3H#SN1q6Z|gWmF(^{sc(rl?afq;5V>JY}X@?vqnix5OCYFG264&Qr>1C8K3*nTBzqCA~mh7)XgN&e7tWU#$4)TpH93=%=nE_T3%WwiltB#6ioW zx2KVyWolv>xe~jx@tCKig^@*c+tQNVjVo&6!Y{PWJxXRv(waCC*YLK9bY0Dq{9Z4; zd*^c^Q8R$m2*m;tXJv8|)rq|nHdt**R z4+=_TI)57ES>s!sY(OiprA07V(|2=Jx;*`oo&7MvspI_G?oHR}K-y|cIMDKh9AQr+ z|BF!F75Fr8J8&m(H*hatu}P_(=6Pj_|D~uQX75a1yKlFT+R#ki#Bb*w96?YmR<|8r zzL(tgUh7@2rC~l9I}eRzODWRzi}hp|30q%x*VZ)zM72h?CP^n=6vd)3@i9g|-KH;e zjuJbvv4ARxS?$h;W1o=w{3P4%Jmn0b=Kt1`QoU+LQb90v-3yZW`k$lh_+Xifoseb; zj5KZhI`ReHPJ)0<(=4YnZqgj`ZD&jLVr2U&x_qq86K#{0>J{HY=~Gr# zgpx#UQC!|VZe9VMZi;Jhm}?aR_j+Z6((c*kPW5-WOtKfKp9QviQt2vI{T-C8=dDJh zb7R|pVmj2OG@V3<;x=i*+apLgh%}U0-Y`lWEW~M;9a){CMP};MH>wt^xp2k(azWhy zTLSg9((mT#w5Gj2#a{npCx{=%euJK;XK^J54r|@dUC#6koB&Yt4)zPKu=a;C)wppJ= zC`-UX zTtOhsUZC#x&w&H}5B$8=as}k4YBDsk)wxRvPcl}D-yh;+^Kjx^M7ekIz?VT#Jlnzg zqM@|(c#9NkH8jMMUMnWaf3u`NZXgWi0f|qMsu)ptXM{7Ufc&E#ER%&r4Gn1edCvvU zC!Up_1Mbwu#dKOyT{q^d8ZGrx{o+@_v(iYz7Ini~5OsE9^ zWzQ84vNBb}x^02+tlg8MkAWv{*2HC=G+#RHI^&AF&bq$PI0#NH1lK#ZI=4ADN$*Ot zJhMG>Jn83N|Ml1mvg_Ehw(XHhU!nRSpe7y#QMghai*hBri1r*92jeMVKS($9SPJg7_J@Jd?uYIa_;WHWzYwPAmOW_SN>-CLRnvVGxTCSI*G7SrZL^et_Y36J zgR8Z5(2bU}Nt`QdwDg&WqT34aF$CD;o^;N(=|os&@&;37ZY(`)5kjHua0rxrZXc@< zL44t7Mggd`tr3#pEe9XN7Ej(U(iEx$|7o}-FO`-_j{{EvPXl@|9P~R+7F;$)9ZUpr z0t`L-83#cL1wj!$%SYu62Z)@rTL^A*&wYO}IrG1p7D*Fp*&s0~!(wQioQ`6Qfe2UL z8@eIPDA~SnW5mo>_t@J+M6<>}9I8cLh;y9WUvjD|{|$b5MzZuT{!{o4voHq}uh~<# zNo&Gs<|q612S!{OEAtP8l5KJZ|10*AMa(Ylgh&6LiXG|1jeMNDYQu7}Rr@ndH+s(_ zgU9_MPQK(SDG|;cb7pfON7(=VP;^PQ0PSNC$G2fP!O^DB8LcWmFHBA?q)#*)Y9eZ* z!te&hD>><6ya=_UlqO}g^6l@4z}ZV2G&~yx$A3d(fz6-cnP}=2kbl`G@PuQ;v@1n} z)OuxeYztLidp?`svT@|v!mT67r+F?XWtLnF+Hjl6#K&@3Fm8}~V>W+PMrUiQZMBC6 zQ?mLe8IgW0mxj95Wt=95(>Cws8xplN&2f7UY1$F8AAZSdq&#c!|Dxdjgl?|KQ)cfyLf(7C?E5Vy*Eu4* zj&n=w!)}DTF20|zK8DPL2H|z=;qZ<;BF-Vaj->0`NCSGC@H+A+eHHl@C&c&Dpzk7M zE(piC_eYwmjC*bof74O=8gkD!gk#{Q{#KcBou%UIOwc!Zjq7|MzRsbBt3}3`Pm}c2 zM8bJM{va(Br1#u1dTud8apfd{P)sHG9d$$F%tGUS&&$GZ)i`Fmu_p8_!ZF?EMHjtf zjM*dmheCNeWU@LFwMIQ~P>&HVp5W51#%OC#97Q6kkr&~?3`b2^|qaJXwx89~xhQLAfY+jO#sgH;nam0ws`(aR4 zB+o@0n?7vym+h@Z)JT-ns(p!Emloxt&`&4<6>td+pvdUehGulEmvuMR@%Z)c7-Lrjv&=F6DdZ|{muV{`fyC0U)V%Nsb2{4+PnyJ$I4inJHK$Ml{HYX(#9hv zc{R3_giT^HQpqV*by;$-g4!UTiFbv1XqCyJaJ7PhLdU?v9U<~^`#=x+wIPcA1xX(h zBB@h3cNq~2XdV^Lrk_Ma_3Mi3Ay{!W1deN&IDl^IAUU5}C_E#Up~c+_(+Qv5tKH>k;S$CN|eD7s-K zXK|@lH&k8rzLAUkF1|_Au zy2Nz{h$;v!DLjKgZj z9wp#O(aepW@9$^s89`2teo%x(b{XG6wB=WeE2!#*UiwrLzB|*}(MG&Ow+^uEgJlwo~G=Wex-~UtJeQC>b(u7McoI~zC1xS%ntJkI0 zQE}=#DucXc-(>ZK%iIVfX~Fndd7^wE9h8iKflCt76^^wIL4^Z#Fmo`oL_LIl$;*#( zy%iwcJtEB`!p3spI?CIITd6Cv5n}yFeH58tiEYF)J;0P(KJHM~kB%#J|5e03+jDaX zL4+*HD5<>o(sjFecNKMj20)g#m;Uo}1i&_GYKLJ`zLt zP|rDLw-N9_b>oS4W-84{xE&e!8nqylPwGQ^utj;UOpcA-)|p47fvcRuC6EUpe*h8rrOHXm047 zH-oK{w;0=8F!A$Y56Wfm{C^J0uh?Jx`c?Y@E0FbOL8 z-9zA+;m*RSO@7RAu9Yr~)^vXF>CvmSq=~ph?2ez`M}E_nI4Cx#@2DHqP3pVq#QcB*y4$^+^7WqHIWt-Yy4gb{!3ngjS?%=R+19kG~{=k z3;i(zz@K7lZ1M<|3pvbTdeQwrT*i}{A_J*K*9+i?XG}0-%EGp}heAbjaK3keH%pTl zh5Z{#z-d`;}nW-7*xF@Z{`Kl6nHpJ;30L4|+HMMVg391?W0jjTCSp#T|Fh})gX^+?2N)pH7YC;p@ZF>hwgn}OO`A%l|aa9{K0B1(eG3f5BcIC(DPP08_`?QJ0ki(w@d zLfIC_cbaB|$jKt02B0X0^JqetP3yn_Sy|OWfHd)F8~=7&=k|6RRm|mu)Ug(QvffSP zk>4&1&nL$^B;Mw6ErrQ9vN!wFmI|ugT`6>C|6%)D%5UH19}HzU$!iI|sjg<580>ai zDdd`BzSEbXUyDKmEwgrC*rIJM6pjsMnwys7|3OfC!)aX*#?eEKIal%bXWM_7x%MBd z+qmkis#{m*IWwbsBrcmuoJe$<$;a59(GVy*?8}NmMP=$rp@S*5a4QA($G%m*H+`#p zZ~4~v{OY`K%)J}|QX{AU9}y49Owp6j5-0mncdO79yV6yIe{>!%RNNF-VogTQmDJiY z2WoS4&cW$CDy69^&ZDp(d@;DD(&Y@)mrg&@+j5J8%Q6sQtQg$mg0zw!Tk7Z!y)xcp zVlF6>6s`pS4gUonwgW|TGDq3vQK5!iItAy=NJDWsZaQG?6hv1}$E?vYi3+d!TJ3XA zSFB;SMD2=I;3r25Dc_ucUAxI96*Qr5A$_O{$fpEzGQYS=uAgB{v0jOzW_)m=ZIO*$ zg{CIHL!g`h(H}ylfPqMc2qKY1omsu0v`;5^$RSoet?fIm^fRVqUwMkiK2^mf22e?v+tqiud?jPV5s*AgBcY4xZrVTpElUWzb|8 z=-;aH^dO`eZ`ww~M6W*3o%D(+eZ6GOULePpoOBD+d^G1wEuZ_Ob-Jmfk3^LbUwC?y z3KJNkXS1>}DmC@7@WbWYM<~mj$?6GLIz7V`$`Wvg3)0qo$w(Yc^dS(M9GsR#;=Mul z?;`r%KL4vQNQC{*GDnVRonwz^R48Um?LfV{3h?hP;HtLBi>N;oU9<+T=b-K@?~HCH z8WXdb&ERsTWGbEB{)+J^!^J~xKYEGkR}_Kv`#)s-Qq=ACd+~EJnOjBRdJi^(;itl? zhj4xybL3O^F4D816uVtqcN~7Y-A2Nm5GSC9+to#hxObU5a|ft}Qp){3Vjc^&Ooudi z@lfL}Rf>nqUP>gpic%HzOz529Fgc{FH~Sjb=_Y>L7owyS_`G4qnA8pOoE=RHwSMpeZjt81N0R#&e1K->{X2{8q~=kK-T_L8 zi<#A>#_Z$Y+i62DfY=QEYWcAU-MKFU?Ki*uN3++%y?$H}DPkrEUfLz>v#T?udq>IT zf5-1WoD^UwwqhEmHBG(hATLK?_jFR;9Ilt?_eb9&=WB&;a)oXDovkw2y-L{mACWr3 z5qbjf8)V+46Q8=s?}w1PnR-v^I=(|@=-k|f;goiLG*hA((!yPZRi6_M`S|2-DfY5( z$ic-wGKPGg-6`wIb!L9c81z@{Aj5e~I`qlk8iT&WTO2JbTD^W<1@-a;>YLl7NzL;l zdV^tHknTCAj%m~QDMBuAqjcibwSMOK`OR<1;WS@betRUD;%Iw2W?MM_yo~;tTzf~s zTPOFs$Uh6F-wI{R?h7)xfA0RC6zpH}UOqYBxVN;m^zEZJ>a145>&m68N!pk_NG0~M|A+3;1=C^#Rxt2ny39mG7@i=m&mQ?*y z!RzEe{eL&^==b6w?{)pX@l!t)54mxalq|m?9jlGz3LS zB3<|I9GU+iVL^CJmcISO-tSX`cz3h!PB^sn#|#e!ZHu?vjnSBw8d1Uu#d2N<+|K)! z`qr6cPO>bxb+fVif_ZZT7He~K`TfX_9^Bqd9%H|wbM$@(zx|U-JQ{X&?6UscZ;-p3 z(!J@1PTfQ=fOjc3IyL|)0r!+(8U`XNfT8dn6}5VTIyE>g*x+gOaQ&Qb{E>A6wd?s< zbBt1*sRTbUaUHLI=8`UN`0R4xX1J!lwhX!UycHUb7x{llQD zEtrcbIxd{_OOcaEtSU>)-h1xBi-u12Cymp|Ub+daz_T?BiYyfs_jh*26=9 zoeBY}Mtx7Y-5#NGkt)&AtJX`K-+!NOnxq|lGN5LP)yD#@vk-@HV|6H$&j?P1Y(>2_ zo$g)rahtKl<*34Q(RT8qRrLB92t6+=rsLt}Zs|_z)~$AyxIq*&gg^2{-n|Q-4WLjx zIf`H%u#F$pO5dC*n)qKnS@0eSB>N@;=Lz;!`)5k(d^RUc81YvB+h1H#sO3=(c{hD% zQb&AxD>)NYe^8wnRA&Y8OnnuGe4t?UiR{1=+@ws)F^N&5i{x>eN0NZ$c__b_z2l`V zd0pz77^U?0Dn%ZAoE1Y2@$tG~X_!|A-bLzSm1?4JvY}?aSI##Sok$IW71|ML`+OA! zjM1JE(K$+2qY<4TI$wbDozASt6li@%>R8kI4qb@oNt+TNTt_x(JM4K>1g@~l6h&T& zlS!-vRf=Q#;{^}DSfr9&u?V2eO=>a=WxXjxs(uwb(@5KPNQ$?`Y-GY@3^(VGSGt;F zRF5=gbPvw;%fZ_oSm$5~8Ayk-68yBWnK5NpeU7Sc?XiXdT*8 zZJD-Q^TSRh)(}emC({@d!yY+%^C3{WUB+THX!&heEFGK?wUHKAU+77nY1mx^Ap$GG zU$5{=PPV4?hQMGfDOW4ddve;>FF8q0tE$AX(<(mi;X~KSpU*(lLfuTZ+8`uk^Fq{`0G_ z+D7?!V0IyAN;KgYdW(Vjh|Ad^UH8gc4TMizY`x=eJEj{Ix7e2~qCmCY7g-Ra*d8J9 zW=uZk29pkdSQe@u^dd}W9;blqwz53Uj^ZxkX`+gWI2U3iR6#)oT5t+dxMR_t##y|f z=;#HgPC=d*DaYO4&%y>`63^by{T6JIwo2Qi_oeO9avQ5g#k5_^z zpGGM>#*0uVY6pwkz{v8f9DK)}C!C!&`aK{>kZBP2<0<<;CX}!~iN~w5loO7`b-tZ(ij{bNF>dVw$shSNWR%PLKzFhoBQ1Wl-d(>cCei&?NExoCYHKT%TW7aX zy|RFMUQ+)7A4CG?Hx4i@1uN>+(Ywe^Bmq)C`RUKn3}1vK>I*&TDiaPwP{dsXu!vB* zv3B;*oN^I$+`wsf&N|56)Y}|XsE#j=k0XJyJ+aNa>&A2yR6wdSqqVSa*}~cI-o_sq z>+XC0KG0*D%?#dx>m7AaNa`J@G%FpEp5PemALC!_67W*$pm1@pJ-8&;5nLKv7F-@& z5j4|=s)fQE{tLdP&ZW*Q=*TkguC>2yUq?kRANH)@L`Vs4|DddTaavQ%8Ly8k#%}Xv z=zC5wcD^0XXRQ!rnSqnayNvjyOY*-8gU={&<*f3B^bCp{+H(EnuGUx~C4o_rCR>_) zSe^%Fad2P&aX;(w>DZV%Ix6Zw1V3sjlCu69hu%DbB*%Yr7a`s+JfD1MXBw@_x)%_( zDR1V^hYwiirWyj%Y$BB+WDPQa zhQK>2e}zO9FYpQdRT%K&423n5O4Mhm%VE}ZCHN=v!mt>}SC_;av-U-Y<+2jY^^cz zr%>x((8tFmCP2xMR#ygT!Fa4oo(<%ESUUO60}oXKDgk$Deq{Xp`)uCw6{=Iw6V!;O z-jixjqjR_?sH=key(#oD`nL+mdtYI>Qam!N7+|v!Q?@8um2HY&ol_B6M_fa|{}NPh z=biqA)z)PlFW&uN_q!A%ycz;;21ROIC@vO?m_6NEMisW6gNiL`c2jsH6Wkw)#(OxK z8!A#oHT?e2E5$byQ%BXvYkFgp5?rW(z(eVg#2E-{4Z2MyWabh6a>ysUzOhfdwqtsT zB=SR`^b%K+L=n-1?cPgm+n&$tSc*wT7k>JfWOVOIRt|M9G#&XB1uG~AU$gJ5=V}3c zuA+>g+Dxzz>46M@B0^<43!LAQ6Yd?MV4}X9_cRp#mVtM5kmX=uU+y~S{#MYWn1k9u zwa_iSt8pa&$B@?c5W#g;i&YYxL!Qhbba&mB`0>T|c6(++Xpxsl9Y!UI#iI0u9vQ-G zpT4*z$HCTXt&FLeg-`|l%)v&tPA}SKww;%06rgL9LRE{{=QXPqzx!k;g+MbY`>uOL z*w3Z@!4RV7y139A4(Uc%pv(SVtR5)4?N&oUBk)HSM+`AD)8*~%4Nn}lqJm1n!=dIQ z`Qyx^Sno}!ttnc%YP0m{)+Z8E__8nq;UAW za9wXpZ4o10_(NiQxv3ltbOw$Ejt6GtU4XVgy*sJuz>A^;Spx;*xW)RoA}_c1r7;~N zxDbr01&{ze?9&ZlAKjL!OH&O1TwpH)&teC2P@b1Mww`AXO4kjl+lRThRp38v=d~Y} z{ZJ<|dof9r&0}k#fmtJnM%$u^q#*Sg(Ks@kNa$CA0>>H%EOxd#m-7WlUDioHHp0?h z$nny=t-qpX=`g7L%uS_QrUzI=mv@m+^R<+&;C~l1DUfED00aHNzIz*4+P!E5t?`kR zS+NBKaPxP<&W#eCwwa9ijTy9i&Ug~h_AnutkNn{7d40O<~>$<0KOzn=h z))_#GSU`w?=yCDTG=(}mMPrI*HFuCl1OG6nqB;;3U>>|aY9O#CC@A!34Z1HzXq_3+ zw(;@=xm8ZNE77)i=&CQ|tW1k}v4fz56GOdV-EAlb-#P!<;9kcBuOX|5GfxBKg+O_n ze&Qe~o$1cPyyvZym_*dFWIF(gW~mkz*BWBBXe26A8*5;WDiEL1#>8#0F>xFHDNR(I zoMCBa>n}-}dvH?aQdo~6e}f7kHD-2%*tgeOB;?);`Xrrd=?sF<7WplCoRWL!Puf-} zdc0|S=O{_!hbA`e6qEC=5_Z2GOdniYL=Q|MUtIOS09AML?&jUg`z-H%o@LQ)#t@wp zJ7S4RWri_z{Agu-1bN|{CWq~FQqqnUq)go2fEmD7IYQv2!k@IaFJq)x07dZtgc^mViqeqf*ei?X|cqXv<4w{Z8LEy>7BnT|QJsD;}s0XC2(J)1# z%IG32QO)(`zQ1|jP6C>+uq%y7N!8#R{x40uc1n%|0V{jZ!z)AQJ z{8~d)jx|JO2D&4R@y{GKV*FU8>8N=N32vm^KM+cA{T1U^kaDef+9+SlGwPYK690dO zAK$ZQs_6e--!hbPps1r4P5i@HhAJq?z&5r6k%Yo&%L`DQ>94J@Wl`am_0b^`0vD(o zf*iJ?9~2qDtsKN;tlD(gh_WZBSBidx1>YLr_ybyOG^Crj&xVDkcvmHfz=Q=W80t{u^qLsSOsKGI}9pkJF08y zde3p3cAjyXd#YS}H99&*6{Meo3SNa!fN)}2r9+{D*V+}#@=v5`henb=8v-trZrfXHP(dn)5b92QTU+69yp2`T7Y{ZwvmiYiQ> zgE5;YLmOgJe3Y?~#DsoS+QUOjh(FJ}iO*FfBn|Mg_6Th%^tSocsr}=-QOAohY2r2n z^kEgJs)crH>N;uHjW7+IHVY2bxBfXkE=GaL%D|IGGgD6Gtpqw6O-j_9vH#8A3Pp5| zci2}EkX{n`L~ARpmuR@jEk}6jNCIdX3}tD0&IQF(njGQu`Wh0pqCQ|lC>$mFNUCuT zhrOoPclLeSGKxkemVx(OH;c!_6!i+BWC?O}c@e5FBkNKnQ<=knmj`!yeCV!N>>>*Mvu7%(<+v~PfTr*0& zDy`hSm7)r#1HH3$L8*Y#niDCZquzcwMOJXiY0CdR45Zy=N;(Q7H;Na3M`I!A@-J3)1)o7<@oBa|4%UD2*;yS>Ch2-g>@!2hAfqeAeOnsgIq40$te zWzj481f7!I91NkmuI$3~Osdt}!X?((^3r;v2z*dj4{RzK^Er`^^CjZUBffQlZY(S6 zS__2&ZMGD0f<5r6pny5W%~dJQ3&eSw_Y-2Z%^!li`*tMD$Py#_CSS_i%I9RdQ@U|j zCp|+mo$sVOM?WGW>@BNCkZ7|80yGqpY#g}NJ`Ach26IB*pfznFdN%6XSR<26nxPfb zUV)+pG^;<`9qhXFi@*N|zq;seM@64@ z9DLgy(~xHKT{%VqK$pJ=DIT(g6cclfPo;IE?&TYAEeQgDFSt2)-E+fp({szy<>5Nh zUxvO*{2l(K{+-^WyKy#@hCtdd7jR`{M3f-WVo8F*JR9oV*UJ;;Zl%!5I#vRhf? z;DSO9u$qzh%hXPA_O@J^^?8)a-o#DiaZ~Gwf^Jy$hj7ot`EQ>uolooLq>8~FZ)>vu zi@D-zM`JnFB^6YaR&?6h!*Nn|MOcLu!N@9|>H~A8IT!QpoqmfZGG+&7yeo}|Q=dS-xFZuNAy{SdX@)=8bHYY3Er&zRY;}50v|^l{d<_PTc0H@j2oAfx z47~+OfN6-`qRY|QF*}`|dB-9R*mv{8a|w>ul{inf1O?DsZNo>>!AOiWKKoF|6u+(< zKWuCZkyTX%`4{b+f5S}a>5@)`nxj=v@I+nc;xrKu2%`oHKlF*+n>efN^~wfix3))% zIhN5lBm8Eq&0vx8Wd-=p`uNn3tv*?tV6_j9+@pCC7DMHUX9;s#|Alp6B0bn{SOUiW z+S(IUniMl|R{GGyA-`P}TvMGRJ#Cwyu!ZHVcG1YAQpemWHfm%aD|UwcroNth`X5-S zQOHMI?~-I`gQaqXJmG$aF;<_F9BX1B>y!?k>m-U8os!jeP~K~~M!JdrYB{v|!V)s{ z?=A1(#`Z%^Z0GPRsa!SlYtm_2!#=ao@k$iycjIKS;} zi2~`>LV#5jFg6eA>Fo_R>K9AEJql>iPh|I!qA(D|Az3^ezQOeUc3ZZ39+^3Qb8!kB zH84mpcI7D1%T>lDUDm4%v9nUZe4{!nvcu!_7b?WaYF1LTQmB!Fxg&MZt;@T{yVg6! z#aRNP@_8wEQtmzDWU(BdWx9P@+IDCw)p)T++}wFS+@Vv}S1Euy_QxZSj>`Wv5O^mq z^-bS&vbIHXl$=6I12USMX-HnovQ0V@rKl^V$QOb`bJsw@$Y1{Ih8_E=Y>|!}nopNP z>r91XoP|5Du92wtsk`CvG4n0Ci$pcA`76Lj;>XNtTA?)`jZ-YB!B9e#?Sv=%={-_x zCh2e|VW){n!yCV>?-gw{`j{rzNvU!1A!9qz4-YA%9=g~73q~6c$Z#15zwz*2k z-xEBQcRFuxaJFY(F#U*cUzFP-af$Zw82QV+wtXlSMhmeV^7aQ=q@vcO!{ta^p<1kF zmc-ihHYIW~?j|SIU@xQX_I0l$DkUTlUjTXHtdP?rZVqzZfUQ%B@QA2U1C6CQXaf60j1DLP*BA#iCB3I7O_Qk1)MUb) zxzS{&ZmkD64+M?EEu%5;GW1R(3vwc$zI6 zQpzB?e66JTuZrW2h^;wBHm{KXXMa-X``wl6jg9yZ{(e^2H*Hi@iH(XXbYb75_B=^i zcu6Pat-8cQw$P{Hyp(zYEb`WSw|hkX83Uk5BsoIZrVYCLI9@7T5G9?oZHt#sVLOUfs5^1_q8axs15QbEemCe+)xSmmu(ybb#i&E ziBMJ(NrivCvO(EjaG-$dF!5$jl~8a>IxX#&4oFWG-4;&Q`8X#rVGEA;Kj&Fv&OZA_ zZzIZ(PP2Ey3v}k9;m1v$=b-Fk?bWYewOfL%>vi`kayI!Ad-?~#wSj{{V|&pWbX5-< zdwM7lVj>rAf8^5;MMdimML)K3AjW)}WEdf;V56;PAQaz}Z^?6=pHlY9jC=e(+l(h9 z_vvkE9*_K>YalQw|4{I7@JR4zaBBXv0J1<$zx>zp+w!O9^H9t_7M_3VGJE4Hoyvm$ z4!%vp>Hfs$QEw<*?-N>=^62qOk!uy#a2f)k7GCZpg)rH7{JIQmQvm{ zuzoLDcYvUpoK~Lis0ZzST)r0q&l<bz(GD9c0e)=;3caJNG)j;4_P!NA` zc#)J)@^0w@=R$~XVnEh%6?fMHlo-l0-j`c8j3j^37kct|KvOmB96Qh-{!X6|lWDY# zVW0g(d#a6AlVnN+r!cKD{p*e=?-(CER_h-?c-6v`bgFannKM9zB7L9-@84zYxd@2e zAA0Umb}M@nVc4s!?37|nF$${i42bLtl0GIx3OECx*N?3h&e0aBwIN2;u0}xrY0i|z z_ceP?4NL6lms(Tfomij1In&!p0NQfmv=qNF69ZV0I1FCTIeMS(D;RU0Xa@Kbw|ra^ z*ui4ZP~S|EV6B5M5^yc*{?O}q@I=tG+J2&Y^MYkYsbxfB&F(PSYdNLr#Pm6*iE^gp zLt{ zEp+a&MnN4Eu62n6rv`PZ9;UR|Yaw{XPW!AIr~mR!$0pKE>~3evx=d!PUAy!OS3=k-tvTi)%;oIPEp0IkF^#V+oK7AXNH@pY!B@i-AJ>> zD!?}xw`RF?Fa+Msc9t-JD%M|yz65KSLl1}7y3fCzhtpDQ{ZUkPVn(PEll`y2 z;3OgZ+2r2vTns^8yK93oRvV}NNQ`LuGV~VcA(?K+mo7`O89IeN>un|HfdBx{L&Zj0 zzYI&CeuO0sR_umN*DAoDc#4I2R?w`$5NZfyxiZF_RSIZV>I&Pv)8oHnd^oRo)B@?l zcc%@eMA2S`-l@o#`DVG|YN`DAb-7LENGT?quB+0MC!6h5K-34i(<+5`m9RVF&QQQ_ z?Q?$KXn1tI_QxH_IL;^sU#IgU+sC$3Hp89??=Hed7Iy>*?NK=y*%GEwpv`(qTGbGz zFdwxLjM;_6p+m~lEAuFoky7v^F|On=LWF^S*4dHs(s#NH!a`K?|6lN%bCe$DAIIbs zvw9H$6EE|>fe~vJ>O)%my7@Ixp7ts{cfetOHFI((dqgB*eL@cr^_udyew&K{<_qXq zoZ?I|xGv?Hdw}!;IO1t_rwy!;s?zgN`Qu!cCw6D=XNilt;dr|125k@bSI~FNvxw1h zP$CPW|DJQFh|^4M;=o$&T;V)v`_P6Kp*BYEej*&RQ5k`MyMqT2CEX^-qxNF7qLFiS ze<<2+>4?45u8$o-62>|xT&Z4lq$Km$M0CmD#3vAbt>XU@^!~tk&?%7A6>dYg`fiC! z?f?0iHks7%9@rsofQsB~wrF9F$$;WyMKa$(QFk$zKhD5Q;|xf7Dpmbb=IUl7DnD>mQA&Y-!zMWB%w;);}8e{KOMysfX%A z>bLPArgbjXmK-`uMTo_$`W)V2)A^~TZjPnuxAT%)ZyZ5Nql2Kd-jPMb2NhasA^6C% zR^>M}uAuRT1fwOZ6XDPCU-MIj6;%BO44rRZU|(oY64jpN)I*4wMG{PrKK^&Aar}9n z7z2p~$|6qthmR)`C`xf6S3a$B`XfE3zW-IKKF|Xkt^TBe6sHkW1aQ;+sV$8>!RVP6 zp_VSiTg&|(gV?x{Ud$ zNVTrJWV6;a)*~td-(wHT>QMQc7b4=kX3N1hSviZOSB`-`Y6PiSFyb*>)v)Ke;Ar@Ye>k2=GZ$9guWPV#L2<><9(~J}6pW%3B%Z-&HlR0N-{gk7m z_2Mv%(!wN4ixv6%LeKs7U3N1wIMjz33}r3KScQmmnW(1GoY@zk>Xlbh&d1gXkJh61 zUmuR!sSv6c<&Vx&?K@npkqbsv^*o&*t#NO+vo@I6duRxR+Vfc3RBn6KM(gFH!%>@Y zM+377QgE8QLvpbK;p-se{O980zk+@lS)&K`hjx*?5&9lz`>2J&>)z##j_h_xHBfj{ zx$U{*x$BvkH!JS~-1nqA5@`0n2(_1Tj>b{uSk4<3I7eQ&Oc{13^|j;LSxv}U!H@R} z44$G)Rm=*KW_N{>-&!cV2I7~5kZ4s9NQ~rnZFbS7!45y58|W0MYsZ}bWNfn`e?RDz z(cTjZR-t0?b5KsSf6Y~jPyk#F1+Dq~#kGhI?PS9F#8vhi%KHnA*tkbh2s_CGz=hUx zwoNBU=jwU?psXwC{ygb2sj9D%5+C23G`KVvH$Gb|S^d0Xf7S3=&k9P?PIvUU5RQ{V zWihzcIVn^gVx6V$<&X$zk0(Y5ys4;%hy)-^OA)|@V245y6vl<67mYl`rRG5pVmyb1 z@HjRo_$Ca+Gz?MGk`IElQF_nIx`A8qC5{17r!b98*^FQal+`=fmZHcZnLTfMfO^+y zxC$_3mkP*RudwY!-aL6Al(0Eom;F=wZTlTNsm~&wc#hESGtzz?dyp{=y?^*Sv-FeWpzVD8;(WO#j}~qgN$H zJHsE(Z_c0Oo$O_$d~GoT=L{vz2oqU2u3;ZYF^W>WC}L}XKwK5zPw}TFsgqS5GR+hg zc~XkyX(A)^23?|44guU>y)KOz)2vV-e+>jPy0f1InSv=-EwD6H(;9UtHkJS*<-~-z znfJOKLnKh9-yIw?)D(mW#i}Njs2E^VYif;*;VmrWdqFq7g><&;{$j>OjQeL%f&bs` zx9?Yt38Ur81KO<2S~|n-L@w|;kM5_^~Z;w(u;4<*0%#rWp-^~{*6BGlqIv(ad%3GJ8*f~ygXh#V$zdHen zirA!_xm+dW@33X)lf+*HVV4d5FAKpy4Y^`gXACQv^#JusF!M>U@Un$`hW+7^A5hi} zgCRt)yJj#p^wYz0_Dp!L9y-Uu)9k(c=y7N0&+KP<}jaF3~G%l~ep& z5sw;cT|WmEi|m^nHmVt-<^_FK%y3f>e)QC z%f&IX`dc4F>Ir|OBThY22)uiN1s0+n%rlCBjfqM6j2I;mV(D#Td-P!?bw*UBG-$B4 z12*~=!v(NWj+Oq-la^{q(y`85GO2P>ywhv^^mA4hm8X49_%xox*1pMxxX(O$vR9LA zzA2#kZ85+;*$Da7NI;(wrDQ((b6`#D9JZYA+jBbgbP8L)v(hb94$#Tf2kfNCzcz(hn zV_AMDi?b=()5-;QTJEXD@PhKf3#O~!#DLw#7V>E|8mZP z{qE)Rlh!fLd(x9FXYI5OV3Ko&-31XPsW0@rCeyU{Po#^|CF28(cScFU-ez5t;(~A5 zwiY*<(_&VrHbiNBV9nt`Di~iWrxdC(8~gX(xoZ68?-FzRoH_3cg$~64*Md(1*Mm2L z;h`^E-oR!w!incBSkXmOsR(agCNioJwveX4s+OIq{qNl{cWeFJ5j$ar} z0Da%LI=kTEo0mui_D}3kHt$r%u6v!Jv3_jjV(PiS`M*%u@gCr)Pp7EMg8EpUqLIA} zbvJ{jT#uAFQdF8NO>#_jOmR$gOmn>MXmd<=yy0-na5!c<9J3s%#>Y+1;36<3CWjB| zK$d_|^&I_CTz8Mw>j#oQ$|@Gm5>;PF+V%c1W;0d?U*Xfnud#yAf^hq0!%i|0=Sdm- zJ|-w+#-k7s8OE5G`b_!-D2#adnc_oC;(rlpKi24qtVJILlabP@VN>0Lhvw~;21DpR zd=9L(0De_h?0U-xvhDSUqV+DaJ92gjn1qF4b*SV!465Ff(_VVk9UpY^)F#pbDxWe+ znvHQe))=RrvIOAwe>e>3LXCFrveY#yM%5>VL(M6BT*^sw38~&!g8x=f)bC(E%D}7~ zBj+UP^QXpjf0%eHx!Jwpa?LQP#t@yewoaR9)B07z)DWrSR)RlM)|=oWHFOS^Ae1&g z?EGkThl8A>FXg;X!JZcP-l=yUQz9!vbCR1@n?ol)o@=LjbOBD*D4E_}k{+M!a8QwR zCFB$B3q#=ixOBg3E+tj^W$1I%Nwuf5wMDQ#>_8~lZq3CPus6l>R!E4)wE8zq`uy_s zeiX~4QfO<#SpcL88!$)Lo<6K^rD|u@5aD4<`g1GJ!(Q}cSQh|D#cR~{~ z*pcc@bvlUNBQe9&74qUk=cu_>ig>;1o?mwn(YvUOD$&?e$HI{Yu_jBw{hQw&)jGAz zrM5|vn&(lMN)OoK$_)72(L9})eh=XX?6ti;`=Uaw5XxBPZqE#Hwn-D-9zhCuWssK| zL0IRw2v7>{_aIF=PyWi2lC^uT8ua?1iR^%O;|?oLt45L)V%YVP3v9B7Z_TBAH%r0& z`#;dnp;sMkKU_kdLr7f3T<}Wp6Kyb3&_;ByS70#1+{&@Pj$~CTX??kK9?}tGrez=K zvD>l7aUt+Y;9>yVvh#v^1$TnHw+oj|n8;&o`1z^^qcRdt`gAW+7pocNyN^0-T@6ML zbfn00a!mEuwzCw<|Cun#sL|Xr<75I~OmW;@aep$^-{=9(-S(fSYDB1OG#E;zI!QS} z1XON|x?^ooH*@T#-Upj8AQ6IE^FAMv0xkykPL*y$vpk;Mv6sCb>H1l|*aJ20BYuPg+4v6MN!Ak>IUf zrk!e}J&W%IGOX>Uws_Qk@M|ZoOJfeK>Pu$jf5@<+Q`yxY&5C9lz0X7C6+1_56+r#a zA^xxMpO}E>&xb5#>uAq_Aw|>s4GEg0?6CX45U?15_!mhq`7eHRIo&A2#Y~YP@)so0 zm`@9~vY%uy{iV4*x!&9WC_z{vZm#~z@j1UF(L{eJ+Uc}1X+<&lU?`oZ&DYu_8jy@I zM$LNB7vZSgSirxBzaOJCD@@toPd>+K4C8nS{_npGVIssKOfTaa)JyTSxne>|f} zzfC$i_JTxhFlj42#xmhYCYl`mpy&JAc5R2YQxky4xsdPB^wdM}s}a7KP+CE&VNK-CYi!eJjvDR6|6 zei`ZrE;4I96qizK%&gK+bh|R8!AW{y72vzBq5M~CjM?LnIGs&)`z&Zu#KoFODVqSM zvqDsn>YbCi;*=2dUB(+>iowzcVRh0#41e?^KHaeajU1%faGt?EH`lFdA z&H-7fLhYb}sk^>L6^T-H2;zQE3B}x<|6^m_1pWUL{ximizVC=S<~qhb^Z#wOx7f$p z$^V_j1^5_aHxw!!$lUbcu_C%nCun)qAC}fiVzyz5;wt^7rPm9W1OIowMW!Y9OV(U#8nd-YfCD7O+Y!!G}S@ePPK~0d0LL zmzGc1Mc~fWsj{$v$;P17UUU`Rc|vMhG=j!F8F+1@InnF!=eOL(EB}&kO!bz1ySx-7 zS2Skm*ziT;I#s! zQMNgvx6yNa$5D@Qov#YVgpSoOHpb{f{bi7MTtiObNcnF7lq^&hDT@{JfSWxbMZa4) z?!W{xb9xKM4BB<$4w)5Y!ZH86Z6>WA;)e_-xXa2h%Hyx!=tCf)d1s6YS^)V$* zJd8&q(jMF@N{MSXj)=}Sn&vTk`anaeSZ3j#ls2`6Df5$3@DO3&rJ=OG`PKAqE3!Z0wq(b<8@j zKq5^KUKJ0)HTqid>?2Y~m|*9VReeUSjmw&AO?}XuYaDv>2(p)d4F+QJT6P~nj6$z~ z{B15yVh+915^z5bQhX~MSJXk_0~5!X@10mqltwzG#P1oXI|muFwnAvw@YK zBmz)t)B^|g7|HKT0GdIIYe)a_1lE4@vgKiH*vd z#gD#8eu6F3+}Z2+?DF{UlfNwjSR61g0XQtAul&JuDPA9=^f`N*LJPD=REkBS6uct? zcTN^_3WTcP7%AC13HzgXT+}4ThK}XnTd!N*@AbU4JnkL72({aD?jIu} zbP2cbqpv3Zmum4A;8QyJWCIU( zgh)kj5R`7uW^@wv=?x>u-a9xudS7jH&02Mq68wJ-$|&6~vt+SJFY3}`3azV^>Xn;y z%LN`RR&{cubQ=QCQ;Cjvxuq4t+~9js*Krc-cO*{6yttFKwNvlQYmZK}Q)rlC2C}d2 ztB5QoP7<2GjC2S_6{b4`sHR_I=7GQ0b?@vac5>nsgDdQ!n2|_$tVx1@{qyz?bJaSm zgKjA=)0{%cQHWWc>i2sTGjlR49nm z7^hE@f~b4-qnQeo2@QY}LmD$D48c&SL_}(nuI#Y!3yodjul>(KdDKZ6g_->pdr%CA z>WE7q!Y$C}nWX=9dnBq+dIc6{FJWC-WxT6U;+0Re_BVJL>Q?6-b>S4huLVU|ZIrGa z-AA)eOwE@5=hDX0Bz>+m@a5`*c?E%o9uGjEpCxFteqtygAL15((2!_Mc)>tCmyu?h+rLEy!MAnE1c z8}`CxIVRKb3o4%vgwkOz3}0l~mP#`>Gz z8dXbe+D=6uW4KPI%v1%Y1h|t}4tWQ>w;>s(X7J%y1#}oxG6)~j9cr2D_WL^>>~tlvZ;BDa4iR8T_5IV1H3K=CEnQb-6*8J-4x!IFtE5K2ChKei-J zBXjg}9@DMXcMrG47&3P8=)xA~Sm#~GJ;!GPs03Oz{h>Im`+h-5h2XeV!wN)*CER0o zb`XEe9a1{wWMXII`(O>1+@$Eltw1qo=sQnVuBNx;FGJsmt=`sP%TYAhxl0L88bRuR zUx(*-2q}S!o_3ybPEbVGLA~1{IcR{WU4^lUB;j}ZtD#^_;p~FV`K+?0R9ja(u6Sk8tRRe)*@?+(RlJRBEY_4DJXl#%u z7<8>^*%_A^s-fWB{P*&?_m5Q}tH7TKS#VhUwYfV!BT;v*b_1>7IM^7Kn1-?_q<7qz zM0h=Vng6@+lBMQ*HuMPE0tZ9rzy7z^B^DpatHm>g;vd1Q>%Gr{$2FY-Y-ACYV4QUC z|HxD|y~Bn6E@OL+Oa4N2rlyAN93U^NKE_hJPGk`r{+FOyKx`Ef`_@N?PuVc$9jb#r zR8)gu|H}2*=DBjyxm(Me%%AZdwdl`UrZ1BpZ9DV<`)Aqs&aU~FqVl>UA9PTERt

T?X?X<(iNNmG0DrZ?8QpcoPCyCOV*hsaRKNJ>C{4l&bO5MS4mD=0O+k@twY7%yYnXUmtJl5j z$<#{{Oljr~<>$(hDXPV4B%I-D9Tc9iZBQ<%msF&GrH&e@x9JL*9d*!+(yw7(I4WDG z%B`<0Q)oL-3SQ3A_v=!$FM(k>2Lwy09?)c0s+7TY3n_kf#NKI3*I04?qc$aHw#yy$ z?Q!pQ?{gbHtJme78wkDmuXCa&v#p9M$k*Mtz^7dsE5*mfDHX>G$ZvIWT3viA;-(LU z(L0K}8b*?h_$8?Q$d1vRW)*cjuSC~JIgx2W>%U*w^pFTGRi#M)j6MsQr%1*k1m5OC zRu7!xn)$BTRyp5v((?_$bBh7moJJpt(V@lsih zAFkGcC45n;@g=Auw5Y$zW)gFJx~+9w)K0d7O7MSJ@KM1!m07Y_R$>U0;glo0t>bX0 zCLmO6VQ9<}2!Xey@FUwy=l{#ydxuAno$2CFRduNTx|K>&O9)Vw5CTMwGs9@cW3N3P z+t}W5SYPilw%7K~-Mt>~UVHc6UGG-6vOoa|fdB!GNJ1o$MJ`DMkwp-QVSpxy3?ioi zeouF+&pCDKRChCbz4!OrKRn}Sbh^T+bH4M1_j_M7FZ5Z_Yqn{=-r?TNSKvez<;*2g z-J)7`*x*uGmdy{jo;nvH4LUj$0WgbWkHNi5v8jF*JZP}oo?>(ylEX=CP2A2%odu<# z>JBa!X8Cfr(6Pa7!M{pUr=hh_mw^!X#6U7vk)6dKccB7WrB`T3T|hIR6P3!z*p?)M z4UwdJ)0-54u|LgKFlo=MWQ-ot7U*aRkB{MNON|Z2|1>0;a{skD;AUQNC`QD*PlNTX6D|wMulbdR9H>JF8{}sVp)Qf`a zGOo4lw*~7#kqDUxgQwcd>>=mI=rdqE4EP|# zgg;%UOos3k29@#+Jt7^V5TF?cV05NA6Qa)>f8aLK+}CurIa}q7szG|?ru;^k;KxaY zhyzxNyym`X+4cnlA}JAU{!`fJCmyvnp0tD((+{79NRnMs2} z>9D10qeTsWX%g7V?PSEgy3^F-Kw!0DQUaKEfiG8I*ulwVSh1X!hC%DV}m&+$Gfy=c~P_D%Pm>vWNftatnbe?+fB| zd#}E_rcv03jGMHPpJJlv!0mI6OvM!zn=~3yml$stgH=GU9pqei(N2=yf`P0=oel#B z6)F#c_!|2q^JOz*YiYJeHpI`+8%!T*1AVonTV}=#nm%mlLEq#vW>$|finOj>GKAl; z-7_>0tl?CCp+9x5P(kL~A*Y9}+m?3;`Rkn}R$K`l4Dsg;n*~RdGfF#a6>dSErCc>wL_qc7=_~K z53o~EgchMyXcHuH6F>I`_6SFN&baN?Cf*O~wVZ0?6l`H8OQRv}$8YV2pr{opf1Uvn z;oD<=P1&e86`mjmi)pZK7LFTEQ2rz>!KYQU6gP2uo?m?jS?fVwzJrLu@Saxq*@z(;P9X)`G@Le#Wweu{atWsL$DK zYJdI^vta#eKq}}n_oib8-eNAW1POQc-pA;=-wpH#eC_rN9KqO}r90Hbt*ZLDny^`S zWL@)j(e1w;aOCG}F)hSc&r3an=fEC2s$ju;gv6=3!Vqw1b^v;76SuAhja5bm;-Tiw z?fw8wzgdwcSK?1t50#djmUr85N2F=5tche zBjIB1w{7tX3?&3<83!}Ci~r?%+o5@fRI(ycL};kC=4_>&j!ZsU9cJ1A|;F^0?XHXFK5 zg&>WE3?|@5n{20~WClRYY`bD^F}IrA%<3*zHTO&^M9#6(g>%YjWjv{E>D{$a=t(zo z(g;YZuvA)x1i+$L&Nf~Z$4b0riCrglfATQx`M%63R@u+IfdyH%D$DQ8$SLl9!sWayIt)HRi|5)d4 z&yLT-F3ViKXDYqE96!@?y=S`O{x|eY|C=rNe~SBZ=e0s?H72od$hWyyW7m6z-VJX! zK2ot|u*NKK{587UPjtp~Z^zlpZ|aP>{QQ5RG5@SHP5WyK@igVLCsS-kc~SNeDMqc;ot&n^5W zLEgc>)ZTpJQh#D!YN4TQ_tqbxQPJ#C=eXCe@B9&s`aoyiH#Fwqd3SK^*}5SV%-nq! zKiEW_iC4P*gx>I9b;jg1{sE2oM=6tayE@L9f7^xkR-WD>zIz&z{QtGin1ip-Zdd+N zXA@-fo9>)i ziXZ>mI@5neUWC`+LOnyerMMW6D%3T%yQ}f2D!oy+y78)49GuQ(Jgwl(DC_L&dECl} zcmEx|;;-2=F^fCA`uH?F6aPzR>UZ>Di~CcZG3_UYp)vh+#*|z^n@ssf_S*?x?!k&; z@(*+_^y!yoAML(rS9=BH@=#SFZD~E%`1>{7+>|}9c5+0nf@y5Jxb@d=-NS39Lg#Yb-1dgngZOg4*Lls2M@ACO%gjsNdydnQI-mgR1!!C05 z&o8_1u%GEngrRlaQKDyMIQ!8KaeFtvKzu;TAf1<-@q7dtldSU*x7A^`Y2||wX~eRDPtd6&Oq59{RSu6>A;>dHT{XY9Y{l{xVG8lkt<msUlh zYrV(5)&}8KO9erC_c!co8IG*^5_eMl8~a)vh7-=~m+-LP>KN&!DeLeJKhg6NwrweG zCez~VYi-)N9-G!*=q$q%MfdTQ{>Z-J1OAnA5i_HXFiZ~6Ll_+jjuGVOVUt)TLA z_O-SPXO@&cz_f`hm`%<4^Gks{B;XGrMuTtP^jG|Eco@ zx3vEUdZ<6?oj#kN;|u+7ovFHfyTG7I!!sW09JhDjLL#%vb{EA;?(U3cd}w@MN&?$% zksPtv>RgBuz~Np-8*+f<=?$xEgF6a?Yc!Xbm-srrBEo-(T+JoK>xP8|_mRm^1kEMP z17EH=f?vj{y+?WfLmdmvWKRWSQjT*iy9$W&>JzCih&NC>Kox)RAyyLjln>oRv` z@9WtdQMhAXOKNSqYDU)@qy0SH5l6Qvh|EgiwIdp0I%hY;5II=Qh>g)p&MzZdY~OHUudX_aNJ5?wJ_8VU(y>ql$2I`KYi{lf;X-~1JS zibu}aU0aObZkVF+$^$JgPLsB-xcN_g=V>I$7>O?6nt9q4mgY~gmZZ?M% zuevlEzjzvx$&q3(7yvPfk)anR=6xn(JwTb4CWNe+B+lj5oGIjqC9Q*MF!x-1#St#+ zhcG^P$tlDp`8SYw8h^7AeZNXhN`Z)a>o4z-ywqA8M|g)ty$$yY3SQzCX%wXFAj-Bn zHMcG(*r*WHs(#{RN``L&WX&*#3M;1?jNXO8>WX79QPF%^YL{I+K~JCRvM;b}-i0c_ z&0WVe<+}+Q$Dlws3Sl}B?0&>HwHd_I{7y$6I$zl0X<;YZZkvfzQ38ndcE63KtV{xN zw$ZIWy;RJBdZTE3O#tqV#K#w~|0A2VCnHTcp8?SuO zqlh{#-4+!Pxg<8Vp%l9uuFp+BkE^@309VA)rF15oj+V=Q2L=)3avt*%5VObVT|Y0l zg$sh`D_v8O9aHd)q0mn)kWT4%YZM=1#D#WN;jk|vh}~#dDQq${t6(XQQY%qcubanp z=8$yJ)hP23kk}o#W!Z9Zir`+Qi1>cK$167JJs#ERox#*|wRAP|9Zoj}R5EF%%>UF_ za+M@eLuQgjLUNv2%NC@D7%}*J zH9T;pgw~E^!d)8#@t;Sk)Ix}_u)foz6v;;P6^M%bcLO3nX|Tg+NL?EOcSpWd(2mve zBc*tM6P&pH`yD0dcy)WVrU@TZ+#cmvfISJTXH`wNX;0GUsd=rqbHgfq58feVVG4~! z#!}N=?x=8&yU+D-o_{{z9&!PALMXuYyD)@cqYVMsL0gkCkTF*b2@;|{hTnl~p9JX= z;gl_CFM#E2W31wk4d}VmF1@*(&zptJKp&6&USwZvuZfxwH8ZL9w@q`* z3}mc%m)L{*;JgvMx=-$VI}h&Vh#2x67<5)kSrkCVQ4LKV3<=dS2aE@e%y`Wcge*eO z9q-wDXMm3BXljEbur0AKwWEe74cpdmNUXC3bagef{u3bcq7G9f^;1|X*Ks}hN;E+q zZaK*R^sjsz)^rb{dSjUTHY_M5^CN>H!QY2EXs|XGh-`GE5cc$M;V=I)YLogOO^x#! z3+XMPaLH;YHU1lX`>PO^>BdlM{9Ou!*O;I1D(XS*yn<6Tlcmv+ay57_{+UKjDMNP0 z#e1xIP`&^S?br7qH&imM&`EqlVIX+g%ypEz)Q=yOA3*c?qk3~20Z9dR)NxHy5#Zj% z>vX_G=AT@I`<4?tk#q2M5aL&Na`5{vwvfqEDnwyrSsm7+egHptWPWV!HH8%4%xFkuHtSO-a6wFAhe9|Vy%g0! zenxwEgCSv=z1X1lc`1KrEM(xltU9dejxY^*N#bK$;J(V^vy8?0F8ux9fkEem3+9XF z(5R8M@lC9{@e}or(oR<<#5CHM+gI4>j-0gvI?z=4v_O2mtd?$Z(9Razc;S^HQ;ahW zwT4*+Rl0-4En;73-x+n(q@9m^IoXAKaYc_wsn)1Oh+WxA8SVr&NNEr?i=WL?B3V?w zRu1$f_hFS-Ey^UttyAXPv5vzeX`p)FqVe8qt2O0EL#hhmt79;iRB>*cG59P{Q%Rqx zoAPlwKP}MEekCoq|3tP28Y&%GL9mub`8qFxuq};+46xeD`3IH?zLI}nnZ{S~)%-)t zBgLm!RZAX590&P58r?)4Wi`iMZP36~%};ChoM{tKWDtg02I zgXEL!P@6xhwb@{kFC!P&kawk zPp!vA>NAMlQ7tp@%ChKHW#X6C&qNKFa$k?DgX?F^SEef{*%04k*W-bt!qqg8YR%dv zEn0Uyvpd8l|HCzh1jkL~D&Qvnw;^+Ph`(OsIwZ8*a+ALfh1B)*iIY)Zmc7awO)?84 zkSy7h;4V#!G73`NmNn5d{{`JZNTuor>T_GrY)TnAWj@OHA09N{DK_=bZ*>!d@>vjj zH9Ur)2_zOxa;$UK+tK+mBQSoF2Z8uK;OxqY&YrJ zZ)c=q79I~lto>oG)rCX$Nq!=e{br;ajU3v1L#ON{Y$d8kM#?3V@rN7(sha4Wkz;cm zu(e&hkb(EaH(?a66i}{Rz$>k);aml7f~IB=#2?e;?I@H*ms1wVMx$N^6St8eTatSG z`U)~b^bTZi5cG9aXa}@T{%A#kE1XbDNL)cP&r=7BdL5-REuc0p8lp-#)x)P-EUMrh zcCO7r@15?G=S4!sVo`F@MMX{W>+!dW6_XI_K+P% zhod2h>>+hKQ0YsKXAa>zHeo1dv?szj5!TF8%YmY|8sO7yTw>DnS5l0MZ^J;^E(NI4 zZxI*oXd&1=0@nYGT?a74G2Am&*Or*)aOE42wHxhm3W|6wtLRG+^NNLaI8d;hOlMX+ zPIrFvEi2pC9sn^ay0RbPtD59Iv;}jeOo*NQ$pOPji{_kCr+q?XQJM0Sye`Fv<0&0a zCgS4U=QvB2tT)Vhw}$B6Pn5<&dTU78*z?@x=?)?rdxnF3obS}fzVt4W%e25$_W!$o z`#pUBzqQ^yl`97r>p@Hc{aL)`Fd#VHd|op&JHVR2fBb2(teyGV}cakvEn zA=vktpuZr*HT=%~`voGaMd9a!XU8+qUu~X`yVo4x^rSdT2l;irmvbw)BT-kP)*9AZ za6;|x^Qx2aU>0&HX8Mc0HlocW2l~^DuIdRejj33IuKAJlJ`7!Je=4YX@B{M|iYq^M z7b>a+9fnw$PeeAmj|h-T+AkHLFl#4MP8-*mNc*8@!5 zImY8ZY6NAl|9JKHa*0e-Bt}CTyI4hir=?uu-O1PxOn@vZ3mZTKgyQH(1l!3-65>gb zArF9?$5!#~g^CGV{B)4+L}=v3Y3aIn@t7nQ znk7r2!pT!*f{->1qFOYIm3j@#|;RC zq;8+fDTOkjNgpMkAZZ*lwDv13zcn;^@&{5oWMh%pSospA#6stJ|zszQ?$xZ z{s8AcKh$#2nSZ|j&NUK%G7S2u&vV-Ey=C5c@uHPD(7*V9%kiam%`ix+w=A>-BCe!- z&(ceYSA{>mvN4+|Baenuk7F3j+Le4WuU#SbQ?|D7)@4;JK9N@ni<%Bm zMI7x0SXOl!8XRs6;0aOYv%tm4rMSB6#r!@olch}84;2??);3DRAxV>OreQiiT5$Rk zB3Sq({NStW_Jt~|odvP&Cf2vN6w2{#S;ktrxiC}@KoKJRVN?ixFqV?lHs@AHDn3Fo zA!aeZP)j^V^T_HFO#l0i>KSQ7h4ee{76}i$&E4VN1RTgv5mBMv>D2B=FrGFdm13xb z$d&@x+T?Ka#IeoL<}@CtBD5x^Z~=4Z3%Ig%xj4f!5eE6GZ$s>)P;+31Naa`e*>5b&JfH&ioLZvuT6|I1&|OyIPzW-^4WGm(d_23r{dkcg_6Pw=Tq z-!S?Vd;)uNRNi3##CY6z4RJ1`Hpqr`Q#(2hBKOc-xt2UxP#g+w;usbL_HS8H#mRL@ zQ`y($>TCNbG8dZ*&60T0u&v8MM6B5mM}xK#^^o(4V4F@t%T4@>=P2*3cpIETkyX5G zI{sycnK*QOYbGpwp*c#r00e!d?MJEd*iU}?SENrs!7{Z2+PIcd z^oX(2@L=t|VqWx|vwMdU=gx;PPO;C7z93!{FNv4MdSRikDD=dLIw1cJyj5)dPsn3Y zdp@pr62)z{f}L(6aMjKOr31*+*SSBg5+%YbIcX^L?-s5K0m2w+2hb{FRIK?P3=8h) z(q6Ut5=1sJP?^=doSV{=i%aC0dM@lPuB0&+!>>j|YC{AO@9P$E|FJy}6>+^%OXoz* zoD{6KZoXGe^p$y~2NqdrK~Ea-rx6>ZYMSFAi;ng}WfcB4 zOn8$3gFt-04)eelONS^F$>MZ*^EZ?^*!~*T&{4JKtKnjC11>z~1{!L5Mz)b4t{aNGM%+KG{ri*xXLJcZc6nrqT`p^Ep~CWFW*jq$A&{P z2O2%-YVUXArbIcq_Cxem@PKNMN%NtHPSus z$=na%*S|Sw^n<4N8yw}ixL}q9VLq@9LRC8q`HRqpiI=_xBc=;Af}WI6Xg!yy5Yb_x z67&~mMKzbX0UZ$K9Q>OKPW5M^z1PoHQz7tY@Yhp>sY03X=er|SKdRv1PYaEUjEl8h zo&QyHK#_v5o67p%yxNVGSNeEN<^IT(n(7$ku_l7;h4D*cvvIR=i*c(_BP5Wet4&VJ zZ?iL11(W3IIJab=>PD+Sc2}Xlx|#IsoUH6YDL-Y8kxBDjs5Bv(cjXH6I+nYRM_4+x zj5$UqQs&|7oM%6eLwST$TopE@yilO5!`JB+q`2OysYXinJ;>2&mIvI#8akt72yeBw z8}=FYBPWLHZXV#vc$HaKTYK?xKVtWz^#|C4J3Yi6yD|-DmCQ_G-!8}Hc=V=I^}b-8 z0gb6Mu*b;FII-Kz{K~v6_lS6m&UKV|c(u%9UWbzvhY1t=GKB*;hkO-l^pyWDVT!3oSl^kJxu=upzIzES|`kdbSn~&G%e7&{a zjGw2DZS`cOISgHAJM(#XSzd$oj8e-SW6F~c5PO(Crs(tzLP&%8y^JnpT@-#E_F6x$ zJdeQ7!=C2_N&&X81?=xtEKkSJ^M&4WU4fVD3ikZWmv`c2KVR=UPJA7wp4}b8FS9WOY%0fHkGB!_b@M+`zQia@H6jaS}pI7L1Q+th5kFcZuUcC9<$%; zvU1Oko`=0x%uA;p#xgdo>~~k>GQ7VF*=zi^T!_zcC;K})l`rsjX4@P4%sD&9qcMkh zPgHm|u_Hd$9T9 z2y~s<%+Crelz{ki|1$vjgDQ4MSv3k} z%UXT~@;1EG&oP&}q0w<4$8}#amuR1s;svAdX~-NSlqu)Y38n03TetP+FJVu~>mxj1*ob|tQ2 zSK=z$P1rAQ#bLopw%wX5w_&rtfNl0=7nQNSlq+o1x~RhBKx;6t_#_}CgwR{xx42KcK^@w(pboR zYJO&ZZhm3@()`l=%KX|aiw=VsTL^aSL4I-LNn@w9xTTR;)67sPH(^KU7V|n8?i*LWhsI3T zanhCLc(pHQt}fwXNj^49E7;~lR?4uc$Y%#19aTy+PM$EYBh5as3Ol^>+3#L0yKq9I zojuQaE%&j*`GS{*1;w?PPl8f58T&D;eSblDir4-^c6gbuOpC#iU~KzwYrY$QJ$B!S zVUFB`!-H)COTN%@r16^s=jH?$P!!^l*YM=B-egzHOEk~QYvxlLceFO(F?X0_%s2Bl z;O~BkIYvBNGaUz%>>$E6V~V$H$~UHqJ!V)AqRM$^Gguj8OCv=iB!n*#Pzy(U|yN(@&L_?BZU#_&>CH zVstvI^M&Z7n8U4kigw^6eEE|g7v^H9DH`8xb77t!o+LG8xQCgFBUnW3S zyZs9>#9k{kO9^^)b?aXy#B>=LkT$v_BpJf}6i9v`jfVX^-q-Kq>);6_=Zm6dyw-Y`i&a)EUP+sIZx&Z8k;v+zvh#?vo;r(Y-4-9U}_Db6uM@c^=`lYhPL^W zF%K{d$^%j+z6HsFZ*gqI!vq$#kGQF~+WK@L&+Bp{Bu7qG1CG;Nbzn3e0BFlO0T z#Oht<W_WIRVbHR46$#Fo zT)a-kLFOYPD^Aqfr@Lc15Om+oNfROa>>GUVe3_T?WdgC9`xRPa)#lp2_Dw&$?f!%? zgXs|Eqeu7m348kz@*@irnc%<7edC?Z5bQW;5Baxh$PCN9n&YfR0hXL6;Jxjz$x|l> z42a`4Jv%%WxhETVX(XiH0>3M*zm3(P>|Dr^Bi?LEDBAg4$XOt~+3sCuQgDZdQHtv| zbee8>e{J0S^Rf+#$Zk|LP1OEBwmUQ{N2?k|F}!@Y8+3W9i~0EEQq-h`7o?5qC0YK#ASFLQDyWtE?Vl>a1BK+qER;yY!$}o zd=1|#H9&XWhuACr>c>S}D-!%KlIijDz-qf>pc|h|eO_1mns80VLk14vz7(~*I|%l@ zlV9mrZ6MZaCd4d*MpzC_=CfgxY7|M>o&aK-co=-Gla%k03prm1h6a_`iTD zrPKL7fpkVEWV^7tVKR1G83rf(U|B0FR>`M$DTCdp(9|hZ=DV?T6z@mA@8;&NZ70>E z;{Di{nSAT*-8^B>tTZ&^C9)f8wk#aR=23SERy$a|1Ok?sZMKq{1NibKW( zKZBt!=EY^8{M9c1Q0TWiknopy)VzB=Muo+CMQbR-YOYG{L3r#Q&f|>E6~EGvhh~#f zc0m7~V%$a+@V)u zJ~)n5x0?BFxkhXhZIn9h(Fq6f2mGlz<@cEL9_E@46}wUkd%j zX?R&uXd`GY(CS%k-tn@;M6QP3pB#$G--*u)90iC6(&Zv-tcgqIC$}`8?GWmu-s6h*95fLvqwhn`{5K1EEoW}=&zcSl!Jz@#i9cPuqD3GY z#?hB~&CP!J%A)#9@77wum7lsyI#B4#v*(Y9D()uJQ%sI>{ZCOlq(&KP+*wn!rW zLs70{4Q5+4OA{b#mf7#s=!0yu~393sn z2>O&;d?SPBibp}p5+mk%)#)vzH84hbIk}J~*+g6(feM40n z#zW>87Ftl2O)@i89;P^C6ME-FuwIQ&zn!WQISl%@h=Ka_Ev0gHgTrM+SJY*r`NJ+> z_SUz%(mcmxaRj3jrU-6KOqAOjtWTN=RK#iGXFNqblX93e5|XqURMfN1cqHY|bB@BR zI*B5a5U<%B(`AB-JIluxd{Q_ibP1BdJrl`fu) zbrP=z(WNY0C@5=W3x4!Wh?)G!FA!%5Zo{G2Qb$4xqiI;vrmS|DMuZ^fMYsO>MZf}f zCl_Qn8Wf^C`E3~3ZmH1=X6Wx%B*(1BRHgBubzoCX^GP(Yj+Qk|V>OG@(AyD^RA-}T z!Gme!4FLNw!)Bpd+!1z7ye{4lNqqz|(NaO$% zr%oweV8Gto2Vbp0lS=iBf>cz92vj~%H&_n@`%!zVi8WU=$<|0n-f#4)P5O$_T6cGK zSou@DhvS(nyp}mH4x)Uh@-7n^eqCNnB6PKAhL_T%WFv-%j0)kDu@otsyUI0S4yDI;!n?t8d_CuiWN#4s}!%TV8A<^3~h?Wiwm@!wcjCWu#QCE56pAUwF zllD_~J(vLH7EXXH^2iO$7)G6%6Cl$c=LAa)%`4}wVaobSZ$XoSUx^n@^J*Q0NX6Tb zec$}R94dUAIx=z5BeY4R%v3H6?RrC+b*kxL0%Tp~Lp?~T!eJUjp0?AwIA1n(Ib1wx z$Jv}Hlt-D$f+3hbDw>0q!5IBBfs&6O6%uF_UNjaOv*%|?Q^N(LUao*9#7NjO)iP(AQ z10pM>vOw1dE7Nc-$tb-CIbXz5$_GJKb)K+OX(6O&vmjQ@Z-Auo+xR)%I5Zqd$yQUI zS_*`3xB192{f518n6n92j6~@#U=*RP{zeY2)GoWQj}VZcu2eS4rRdO=IVlGc_8J2q zQE5@hegL1_uj+B736TlT8ek&|IA~H(40I?e4PpSY& z3u)4DHI`d#-P$=Fmuvb+StP?#t(uPM^uzBHw3w74kxHD(B%bN~0~#CV89R%g%`*j3 zX>mZG&hn%(Yq=F^FJWV`M3T#8f-xcqY-j9*@`I)t%=X0S*Un&xzKQi*XG=X%27?Za)e zKx1mI-%*fOVW|v`QK0MTeKS-_pus54MnH0xDHu3)h0!@@GDcergT%*t;07B?zhO8e zk|lZ8xl<%3K9$`9ndv=iKWDFuIAsW&ff)-KGt8LcIIxdNqNBdDItztd8|76v45pJo z>&K|-%W6GS3W+=j67KMK`5yirVp1cT1E{XJQIM*}P_r|o8atwA%QTLOMvcljNyd6x zvR94j86PYSLb;7e+I9{#n_RMXgHbGeKZdW7^z-_R@Ijd2HQ>!&PAZn73AGS(ixal~Dr#^M(xzvxjmSZ1BR5+Xlk+UPa zq|FheCs!;-RXP#Lx!0sN3!8;POOeHI$2G!by+r6F9X?ShpiBTNcT(i66Cg{YRjIpd zH2z_q83sKMil(`H8wRqn#5zRHr?@9Q3y22%ageFqSM0q-rNu{vAkQFn4cc`K%imf< zBV%1LcK0=`gJTV?M)Ev0uT>`QG$MFqOI#b%x&$8@R%tY(b_cIYimk(UpLG$>mjTj+ zNbPA(3q=kMLIE#s>sxq2Ce%j9LuOzk?E@*w7!E!&Ru z&ayqRR*m@yuX-R@+)(Kc^}^o5aR~R04YuVo{q)m1lu{2Y=pn z5>CPLh)twGw#n4e@ojW{tLHSU6PY8;|2Sv2(?A6L*U`V=CQ92 zFXafJj892qtw00n&8f?3iaBCky$gc_lzSOv?=y{2by&|fp4k#p`WSP`$PS%u<))3e zgLo0|S%3vGY&Tz9Kh)yKoysi4GN%+X+sf^??_L{*8Qa*+*o}McqtHz`&2GkiA|DA|qP{TY=ce(>xdd)iSbF zeKMXm8wkF&t8tHMJ&eQ2gjRUMj_oeaNqNA@$|g*Qv58II za69idPLSN>8FdeHpT>^Qaf+GQRwqp>>ck{HkJ;=BW%5y+XF1Ev#n@^a@^MaMIXkD} zmRqs$sALlhEpa@?YzA!Zt3LaxlnIzEgiXJ8ebzzDTT#z8Le4S;)21nGzM9YFN0_gM zO}+Vk!@a%#ipJdKq%_SK@n?-Yb`+yLld_YUml4YMO+geV$|`mTPla+Eckt{Hqy!L; z88!>2jh`De!1xey8&hPfQh;js%;jk5*|itZ5h^#?3O+!vOyXE_Lb&UA;?92A96#bw9}szb<^ueNY=`I($jym^VfAn zu4wf6TpW~hUtNUqzk+mc61*Gg1L}Sz^-Kl(s1$}d32bXE)C7`{X)gi*a|{mvHQuAWeV)y1-}+&PeXpBP%$&Jsgtq zOo0)e^f@ygG7)+E8{)7Ra!|?XF~9#`TXD2Ut26E4_fHZXDoN6K81R%2aZOX|-#T}Q zU3cIq`ml0;9mRu~uDTZjQ6{Y6z`@b2wf`4LD5rl*!TqLQ-d@H5>yU*e6qY4R89EK* z6wAv%u&)+xm~NVGnYs;X8SA3YoF>5OlZme zl3|6nut>Kp7(ox!U=>Q1|J;b8`|i;1K;?diLvj^Y%}wW)Me6%`f1g*Kj3I)W#51MV zIKb;@Q-~UGzLgRoPlE`bg3(@UyVYl3EGMRLmAH=A#CiOMo4g{@m2PsMbwT?+H_>dK z4e<)6i}F z*=--VBcK=2J|UnuJ=G5|3JaPASZrnoKdpT-t^$sPWL)Ubn}||bbQQ`*^kQQn{R8BG z1?uc6lgD&9+}Ix)2ys=GAxwTM+9^RbPL^b$$<)}MO0>wPgXE|9t266fG7Y3|`#j54 z`!)M@d!yi=5jBqHTQE+GV6EpCaxeK;{A*qo9D-BOlcrT4eSJYhJtg9~e{dBjgtw3k zHb1;?UF7;m$#S@kE7{|)Vn=)s^zn&nUyE)Dqcfy54LjbaOX8d>aPV)Hvh)Si zO@BBw{}cbnPd(?=9pSFfBis}23lD@%&27i{kdRNLWi8m5Wc~<(YT37sZpiS!H_V`fDiR%PTkMoNLElkgAZ<~)o8TzY~*U4 z2oL?4G!X{-jDLs-UAnROEf}~tdPl@99lFxXbQ`rHAMF}Z%7mDGMxTtP`qNWIM-?BT zLv&P4#(qM`g-q&rXNnM0i1jqa7;#>>TqP5kwBe9Qw-M9v_jGxZ;*^2tOt#&&-?5W6 zS~V|OV83f;`Y?3UiZ1BqXR8`k#-r7k>RGz&!am`sb-i&Oe?T}$w)=|t>5fG%qH~p% z{e66IBH9VV*)#LZxUUVDqWVh1Akhb+^P%-Z-~&J~oi46uNiO!)5MjhWom$`|J17yy z22D+*EppLvbR$`XXUgJs-HF<}p=~1eGsi$0)4!mT3#a&*Zkf2*s1TfN%bv4PU*FOn z`^^(!FqLQx5(ZF-+7XbrJ4o;K{cD1IvjX^@@4z4`(GzS-tG?hENPlPzrAA7jI@6i) zuerIet6(yo{zP?=wnQ<=B{(ec*6*ep^>u8i=?UqhtX^l3H_3jFMBzPKZmWT@<;2yd z3tKM>xQaRwl5r=OKAnZ35+B9Gq;a8k6KlFlG9Y@T-OtLbPhBxk7f&PBSPFM4L;ymC z4Uu)FpQ8Jj0-eVFl~*oIkrEeZs8z|IhfmG$Ktscv3kV5lYoMXRg>^Cp)qgk8P=3?h zN_^Fc>b3)QdjNm?(w_($7-;0ljlV-92Lu`^6dm{#A)%8EaX+cFPP0~d?X+sBxI?(# z;3PEKCqPzr=#T`pa5-E5V=vbK*vl+QM^=}*2ct3FY$p1qgDet`k3B+lpl}FEHmYRX#ie z)!OAW$#OA1Bn^-@0VdN%3c!Sgj(v#*IQrO~yZ6GI0cGB>DgnR5%)C#+ygy5uYE7@%^S zt+3qVyn~TzuXL2J zua$Nz2~^JhUc2+pVZBYZe2h@9yvBN)jcjPR}=?Z#M)I@?-QXl=*V{BHSR)1bn8hoZ9>yEFR_w=?9bu zrGg46qO$;Ob?f*Zhr)SlIH-QEHG}Qg-0Y6K=XWtSQ&|R~fYBVONgecLXPPuP|PTkG7$!|D(|{q%#`xF z{Z~ZeM=C`2m}yuCg$Ef1{phZ0^(t`6oQ#Qte0}@=8KiSv!M*AD8+8vSqWL+(95ZrD z_0><&cA3gc*%03>kfcefd#V|ISX+IC!y8@n_Ij{S;@YkY*u&Vze)bLL3$ZNnBTgC% znSQGjtlwQ-Q6373_k+mCseiyq3S8g27yAx*%qLEt_Vmg}=*_2jd18xV0{5y@>w{(T z4z4_mg$$2y(nv^U^U|rWtbV0*kd}xY;YY)t#y*RE9*c`JDxejWz_Y%Z`3WPV_OXm8`ro3v6CR0#Yw zWN!)ALS6NQEGW+FZ0Pe+mx9Zb;c7-7?K3Hz@dDy)e)*NAzeSn4H1=!GG%e`ae*>fW ze#O4nGj7X*?Y~7A8!Zij_y+4{;RtJwYB<=(L57Y`rbS|^sqBVAfASDxbv>%>!4T;C zW0e36HJnoW7zgR>DmYahf$q+E-1I5#{^)qk0I7M8c2wk_nBPeR?#6Osq#}SmvT*~)ZkDA@>?)u+|9pyU9^WIVyRw0eqa>8=oFnU zeEvmxJcyO%&#iY&G*+I{bo(B>L;L)d;#Pj1`7!s*_}u9Eci9l0bLR6T zY`&qw58~^vKX2m7Wsh)fVk7f9(L1ly55Nt-UT&D?NG3#)Zhgsw7@Q{$ z05BdE0yTuC_BtF}c!8*XkAx%*^p*b6mZ{N~U9F>3Khmodc1!xQUM>v55Wu2PKuBOqz95S*A_MO{A{ z(puD1ifo)CrEEA*w^d0jH|00VMD2GtBr^m90wO)Xf_xKg^BBvsU)%ilacFruz445$ zg)+Abj;h>%O^AIK7rRmKy7G+Or76qTa8B?kb8GeA_UJf{#n__vLV?_3#T~rtcGzwQ zt~SW`*;(*>`4*OOf5E&?_UsGbmEZ!F8 zy}r4kX8OOOdws#4=PIQISAM*No0>7BHlMmQ3OQW=!OX6{$m_<9(l^)@-a`2_J}I6t zU)|ia>k!Urv8&s|JFizv#-D-$C8}r&%~5skX+fH^bE>C5{w1!Zuvd}Nb`SRqckpZ; zfV^SyRD}x|dh%;4X5&v=rk7zZ#$(v@5B;te*NsOHRKUE>xKj@<{5=};l$}~=E4+`3 zL|1rLSM`UtpS9wyYWWuPLj9k#?#H>-66Tl>?ms^BCp3nA9qYoXTlj%?F<&h4vGdN? z&~@0Zi`6w_KEBR&UK#{(s`@Q?R_@}?cN4>K! z`!Mv2H)KMvEUDt$_=x$;V`dvoddUci?R_vLYz8oC%t**4Nm++jTBRGM5fxUkpw_U;e_aINV zX79Bg?!k%TKGG0Ks0qZJL4mhZAo7%%>bNs~cgu}j3}>Lv5#(u`&}Z?a%Db?Jw+K+F#mV*@F|U ziyMBe;M}TgEbdaxGO+u3I+ZyAL8?zjxhWqHCLLrGS)FiA$D^bM7qKjgsq5N8M!Lb zuRuyOJq?5YINoJ$y?j|{!rWVvb=1_veBp9FCPEzT8Rl_|*P7RvNjplXMEj~vU#eGp z6r{Gixkf=6liAxbOh%NuGSAcwc1%`K{#^Dl>sLk|Q2{BZ$Zg@))9*lb zfK0jOL1?~|5s=iuALWnn$N3X{C*P~Jqy9~~+)v4q&FJCTr&H3g_EIc$8!vqb;{)-Z zz7Y%RkKpT}lK%v`uDVk<+oxx;kw=8i1PL;gY0QA=)dob7MK%tqz#HihwbH&bs?|cJ z>LM#KAVv(Pj-l?gPlu?vpea^q$!;UuX3wRFx1~s~(;&)MP9>F}Xac8!KKE&dRnz8c z{9#vP)bqkQC^&ymWuC{O+^g_6p)!@)gcqi2kf_?1P_r<$2ew7uU8B3=TSe zrr10a_q-~JdR)XGu3**(Hz+t9SUj`~ovmnL*rP-L>_ifURp9 zM6C+nX;2Jav21^OkxF9>g7~xcbN0E`dDdQKB0@xr22!Sjw8psBNQbCfqg=(4ar03o z#C8j1W-Ud&>BE+uTnxLRI+WFfPb)r_vD4_|(mH8n424G>DuNz7u@>E&jNLCWarVr=DeLtK(YMC`kO#APt8kpITA_L`fH5mN;9~ygU^z z$BCD1;@?P?C6DYHWg&fsU04BE$i>o!U0$+Wg`Ibk$4<0~E4fwNYOaY}!)e$AlECJOBgjiM zTuy;mflU48ulQR+{yrU|JjW4b8UK35rc%tLXY@Gt<?@F7TJLgJ&h^gOgyt-nAV$zwvw2U<4CP(QP!Fa8}vQlLAkWrmB0 z<1;{-Z>ta~n3QE>H(!el+$1T(ZwW8LCAbVv!fsi85t+Iv&$GL1ew72i-I-z7u>AtT zLY1YH)e{T3doN4V(C13=Se;WwZRM9*m1z)}7kAro$I@fCYw6{M)0CS(fZ?rCZBcI~ z)yS%p1W;pDODZ4{<^(!Td9)PB;BqK9 z=pFzu^(^>?r9d_+P6bnrXMx>EU{n=$$8gs`YyvHN8AET57T)!7q3z=|q{B39FH4s+ zjn<0DfqrhwnrQuFhJsS}(%ERhH)@Lf!M_GqCM|Io7z8{ga#=JPiwh~jk(w;{uKT~>%#h5pWbZi)5u=t zyKw8|$Cz>HvA0rrKFl4qI?StRXj#<>ZY&=WFMaHHu}KZHH+Qk=XB*&)-${*DzmNZ& zy|a^X#y1gczy8e$OQ+@U{L5C&ikAdoK4w0DG)&V54uyWLu>sgkH2rXcAYSVlAt#kP zuQ&+Rrh(qvjLI{@Z!+vyBYKmMrD4#&DlkfyRz_tn#`3y50*MR3jPLnOBjfEDUBj$~h%iRR-BnfQ0MRLHCAGNl+33)qfpH7PF>|NP7Uj-EF|%7hqyWrOM?(O}n$;@BN0hx#Zed8c^w@5&;AG`G=!jB${u zmetp%wWSKz6q4bKPu&Z!oV`JM2L}6&ZGQr=Wc5OM(ZIHLgHarJoAf1hl zq!Uh5F*OGg{K!p4aAC=p`QieUNvq_fQ5c08C}T}&B&M-)&nDr)ioZfDK2A!7$aN+f zI7*qx6bN&J%DuY5ojS3YCxoEJLFNV_B=Sv~8;K@B7q9aVYcHZTK1oW2s5Z+{llm&u z2b=^GgKJy{5oqD%qGaqQvyZLj^2Zn=_Sl=^ERMTpx^H@5dT4rNdd%|Tm(ApQrlF** zn+ZPq;g-%jdHBi)I2Pi44qxt=c@-a<^_(;Y(i$Q{Xju#a{+e)i^3hTa{(V-nj*qkkEJc-ur}DU3Nb!+Z=Aq)5&;g zq=VF8)fQr@RMTKcD2XeL+h}I07SJ|o6r`1?S1~*7sScm|f>;shPl(ZB`dG+dZC@6$ zMBNsb1O1wX&BAfR6A*3`pfhRjg2`N&}JX{!sN%Y9-<#bn{Yn0FWb{VbDq$iKmQT zUcuF%ky5O<%E*X_Ga+V9v@fTso=9!Uw#=l?Ipmot+zN3_r!%U{gJcqdrxe|PNa$}KXw-?L~KRHHIF0J zaZO#cW1d1lN(TkdPd6twPd~vCwVMH$ynl_E407tB?6OS2813R+tJq`JGzq3bWU{|Xs?tD5LDzlm z#KFmU-K9e0)VQ*^6|pO0R}qc1D)Ht(iu30EhPL^WvESHx?{h}Ry-h#7?f!(2X3K{7 zBSLwcFWjTUQ7Vc=`}3KpVvo0O-J7F`_y1wPaQ^OUBjHD*pU`>oI50hn+GqFJah#e%;Z3B8skoJZ`Q!)YjUNdq>eH6pv&%Uvld*-?9ut;4+1!I8 z@?;&4@U?R_PtuO1LF5|aRCVE86Wk|&sXNMRQ7!qEoLu4}PJv8_@n@aY4l0F$eH*ft z*+bOTvK+X?Dfu!HQG5-C(H&)1|itHLg zpkMDzY`&g#3cgH50!+5$6|wHZ$v9fnoRV9Arc6_a;xF1!#|C(sB;KFgiss^S852F@PoLVlym zi;N2-fia}K^EywcJiG@v)fUgk41MX){O*GxVXslQ^}xK`DC4#dlV?+DxeeB26r4~9 zER2?j`JiFWE9`z`W721eY$io3-fObFSon|VK(k7Npby<^NygL#uW+lr#S@k=`r1vB z^Io#rt*(yq^W&1GNLtl%B&7H{Bm;?I&2WV2HbpUTnrRUAP$Qd2qpotGKV2%Jo?k6K#E3uNK5%x*-=H{l zzgH;gq=BMpqSIc;Uug^5^6DC@cYi3w`YnN4F)AKRd7^I$Zdp1|qSqxzZ$TE5Njs#j zR3GT%Ma%=sBil5-im&D?jgN)ijI$a}o8vc&J}Qr8)<)}cD_Pap#ZTEh8QaBlh*}LA zl71R%nF-N~kx?e2L-tgN*d;9CNkw_{*E)C3GZ0onqoVtx1*bnD7}fhzR+z#Yj)3Gg zKFDD~d%$ReQ!O_mAh964FuW-IxVeFVu#o1uQsy0sEaFfq8{$c2j>BP!wT7R;&*W=) z&1Os0%AlJKTOZBGEEef%kukL&Rg_$GQ6Yr>(;(9KZ3`_$7QHhyS-bS9x@doZRwIV# z6u@j(3y)o%p+SWOO`k08%vHHyV6~N-HMKc$?nrq$j>wGC`!H;o)F>^N_J(6nU9dtX zRrnnO2~%vo{4%Rbv98m8(BU$m;r+lm%YG`{uZu)A=H)_;Q!Eguq)x!cp#USj&ESi} zmV`Cf_D1cq?YAAU9klswMiFYIK~y0W0c9&G2tL9zYywq;N?{r%LUy;d@j}<4>{!CP z>)b}1qV5myyU$t-evl^ZEb9OyrRMTOqzK?BNZG*$rC_0#Xih-4LfemO7i;3g|Bx(G z6sxuEBX04Kw`!h?x<`s#?>njBiAyL-n8zgZ^I3)g|Gwa7FN`cdP3^ zmJ{bzV}B`G%F<`TrwDdPJdHFMT2@4Gx zwrvv>9{vnXAb8Ho-6l$O$GntlygL~miCJLZ!%j$;#K#-Hh3@KYpzAjJDww`5ir(*L zir9GwD?A&d6h_QMZQwzWpoSUTGmsid>df6x=-2y&wi%`97)?o3TxwZ%^a4-X(y&*Y zD$%tlPM(5cfJP}5qI`bRYe6e(BO%5bz z?j`gDt8JZuQOU(ihi(=g1o3`fX};kosIVuR6nZAaC`R1gg-;JkpF~H+b5G)oV-6#B zeAIeIItVd3;6z>&CA6LceSJmN*~dtY+?fTiGp~D2dmbf``#h>UWuN(5EwOo6&I>7Xe1I z00(rtd@3Ufr3w9$w6!d?S~}vC%@cMiEyS!RgY9>p5fBrL)5R%H7s2R>(ydpy)ouAa zOPr`Zyk!pliB+=uOB*FoPmycb!{N#bl^Ge?+a(>SWux*~MB6RLAY6{g^reA(n7JL6 zoqhlq9fIh5iD|~dwK(C}hY`o;i#4)?@LUIiJwJ+B*S8eQsm0hWjKr`;THQXbcae*q z3q@6zdGZ6TW^W6;Unmd;n&fh`3yWbGJnKcnmtbDbIi@D#MU1}O2E*O87^XT>>I0_Q zIB%ldvbEdPP=MaabG-FeX%A_OVE-R){~g{$mgkGZPgO~!(pSM!Imj3=a>Rk0JI&HF z({p#y-k#mk-raY1OL8{GHYS-Y8!%v!$tIUE!DJJRX`&1?IhfE5 zO%r~PBkPSBz86@b+I-_xb=A z7vg-zdWD-F@Z0g<#p#S&^YPS8Z5I4zg^_j)2`OPhyIM=l1BfTF2fxkPs>jKXa5&k|5SB} zS)Ha;5Hi4d!L`*!mYmKvJDy{HajG_0qk7o|klJc7g8owg=G5r)na^Ljy;9J$W&8eR zhxq2qG85i|(VuK{9`My>6XZ4&bX*#Ibw=}#h<`z0B4vGm{`*3wim(Y0t`aWj85Nwg z)#{9&hG1tLh#V7NKc6%OjA(@F*M)k*Zet)#a^OKpGU%AdLyCgBvfdV z8PVt{mP_PP8NnsA^^T#C6s?xuA|VLYB1H^hxV<+{{T9ky^vCc3bwG;0&Iti8@JjI^ zcT#x5of1w9kGUt@Q!cvPS@2+5Z9E=b7aDC7j~zXI)A4A*&?fvN`ujFiGkr9q-{?m> zhW?2m;5cv9@GK}C=dhR*2g}0=oqh!2;-d80R*6BcpTS z83OiOCXz_6{wh$50@{EO!x>sCcQq@16HYb?Ar-4Y(&W{Z(y`0wUSsmXa*q*6iF1D4}{!6y{6)Awr>0!TR&6l>E zEoebqy8s{i>+LgL2{iKf-Hn3`mf@=ZV?l`)Y-~idqFKg9d5|(EX|KJ8ujM1Nz8>bnJeUtUbX`8AL@F0&+))N+ z6eqaPv9rZb6tj^0$yPpUnTh33C@|3waNIO&dZU*mwRQ>)uya5Tt#^95H_1x1La@d2 z4K3?+xVj-=pJ^etm|}0+3cR+h@*se7uKo7t9Z!9)6>Z0zTnx(jy`x-UD#GIRa;!a6 z>inZ%IofwrUiL@}JyJX9O^1=*91@X1CGr_orED^O_DPWYt+diy&cBo*|Eu65|EuKB zaLwjb=2uemFDY53G?3QG(MBz`(-c5zq$I>S`$!3KPB86sju;{^|zdZ%ff4=TG^ckt|C(BNRW>Or>E&+E%MHQ|NgUAS!y{f5z88C;OZ6S#6io z!4|p!^pteFJ`)n6dj=vx3-{(M!41?uVU96>**+V$a(}25Hd^2JHfUB5^(eLC(iJ{J zKPeMji#&ASgX-TGhP4T_oJ721`X&d^{-bho8iiL%`wuxScuTKJMd_uX~y-ff(=50=M4=2=U>%4;-h>mA57Qq-jX@^7zP zTf%7^-kIu3lfN$V3q+^U;Pz8mOYULWi-Gc=!|%lwaC>`cI&Hsq83!}Cw-Y`4f28qr z@o!fkXy)hgM97(K`6#M!7%Rb};uJ49OUxRCB#qIh47i5gJNqG_Q(Q!)p$_>*pC!UoEEQdq;3(XC#moi@IvLqGRQ)}CZ(rzv!R*8DaJe@>oUX+N^Y0{uHtmm7#mI}0Q z_Hv@zN>y(9)9;t@HRVpDa6yU(KJ=?Ar&X?Pq*M?{pNTuMlHfw^4PU zY>-be6jZ4ft49@BgmX_e;yW8|)KjD*X>A`|>%BAqa#|5gEv7r57LtOvm;IFg$`!BT zQ%oeYKNOL*nIrO3S}5SNOtn@ea3^sxlgq2%RDBPjy)zO9YTHHB26NZkb?~m4D79wj zO*>Dy=Iz2~-oQ{4w#0KKA(*hhoGn}ZDj&e97Plik$|#>X}$0 z-gEROG1O?&p@o_Z+rdTL_Di?D1SW0-q-jf28qFF#_#y0m!X&{m&|KOgjP-667o3L0 z{doENFe-)*09#7mx^N@cQXC|JJtH9PqN~NV-W5p?&&k^~3F4MpV)6Ngf%uf`i-^P7 zl^(I@ZC1U+di)-Y(csAAwIbBCGDk&lW*W^=>cdb-=64{MX{^ZDb8a?|-dPl|q9BFc zrZCr@Eg?_YNLRet!lDyMt*Lb@d?sAg$N_n&XOVTWRVxoh$1R1Bw!#+0=3a5#l5R_P zG<8dI+m0xi$Aw6qKUOJGG-Zv5h*fhZU)||X#yg1y+pR6?&HWd31}dKfgwRsjX^LRM zw6pjNm>N_FcvGZQvd-M1FGCmZY5S0f)+a(vS-3b7Etk-xlGWyD*qj?g)ku{OZGfA^ z>vEY*`xU6}&~R;4_?ly;JWDQg%bfR&;fWbC>`n3dP67TBv zCo7I{M71FsvG>1*vaC_O07abKYjq?`q!>tg(HtA)gnA!ZA1>rp&db0pHXL)qU}tC zYz-5@@G%dFl?_V-oWC-lrvLWv%98&=XK^uSq@s6@D{GpJ_w3=27NI<_-`RSh=BPp- z%QbA_B3IFRoO3(F@USE{I-*kf!!t_2jDnMVNFlw4$e%1R-%BIT}#R%tjO>@H}j2oNlwEO0Fs>BQ!sczWek9%nzSo}2UPnU%QbHdKBe z^7R54hO@BDxh#m|Qm;G&Z1-)r*e*44U2TmVah6Vo5m9q@Eo0F*U!gj9jEO$(dChKY z%U2(UXcV~()fu$~O&tUE?%yj*6%(!#8w)#eukZa5J=^bx&XdIw159l#QO!Yr5Mz}F z0{ERp&ixKnGcOYoNZ4^WUYwDsmMO&7&WIL3aSf9WDKm66Qr%UuS>;E|f4w^KS|A`} z8Mu&%N|Y3RG8^Owf2K~FA)T{DMF*)F@1x@g(lXlNAlpOnmJ&U&{p<{kXv;#$Iy%UxBioh`R z{!-*z1G0daHf!H^2@DTqv+^MEczJ_>yhbv|CEoU(K>`Uo z2y9X9neZCX3V{6%4AUyg_Y?5yNJay|(&YIoCdaJ+QZ8|qxntaME=q}s;tD#-Ikik7 zDp3XGv9n7Pz45DFcVJP=CE>9KJx>d__tFKVk)>XUIhJ3tBZ#lNF6KR$mcOYpVdGcI zX;VAa{ME|^_~iPFShpaoU9%EPBvx^;)o;XAU*Zd^e-PuSwn!(GD(r3br!mkY7SqGP zE8HOZqfkP%SJDsr?A4xna{CArM6~6gpB3%UqBNwl~6E=sIvraFtZ^ zXUbHx%EPK$Mg(VTk;181bxG}4aN_Tllo0WuSsshcvSM^isNHWOWMe>NEL>-`GAZao zo;|L(kbcKTl$=BHnqrF!wb$@&=>y#Jw&@A-w%2ohK(yC%|DN%tS+A(ONV=AhlK4CPcJbXekn1 z?_(GYSSA`?yGj|;;R|5K&nOuD)Yi|gtfe)72qP{gFZb+n=@~zUd#`hI<8lkGj*Wwi zsPtzH0@RBwmX@iih;rY4cw|59QIHO4bC~I+Bal!>tgiHv52?}peuXEB|BA{GXqCfq z4Q$V72;UH}Z#KV3D!0F|D`uZrmg=orU-}6Dqak2lsWEveR$MApvv-l|L6n)u5U@W^ zAX``T4nNc&YL%XFi@LB!X;`dfn|-@|hh0Y$q@knvNKI8w$)jux#zPic!)#ONgjO$ZqjA@VD`-l%LR7Jd}zXy3pHm2VmJk9Q#5NB1L_F`a1 z+qBlAkstGoFE()mkS{KOZ7Q`N9~Vzn@=sQZ72=d&95&)kWlj=0xYjb=_F<#HGaVgB z<;*eW>-#n%M}D=NXQ^Ws^GoVpC^+HX#j%98|9*3DZzC=uY-I*BuA8@4WTG*3%tzyW zN~MH;ti;3`^U1g09{*P~W;64*?9&6+2ca<=S?b`{E^hIXnK)il{*iSd(-x%;zoVrB zeK@|cuV1>`OVs|nbPal@{o;&G>BNY9NI7G`grz|(8TvHz9F%NP384;_da<_m!Fa07 zn=7lvqo+C;`ljy7l8&oJ`Etg`7IT%TIHid%?VC1X`netv|IN6*r4pNOM>FF_L%Qy; z)NLl>xni`Le9W9VTPhxS(W|-4oTWdVKmGU=G-fgL74wR63(w?DCi!<@SXIijl=zi7 zO4G`3VW}6Uy1eCh*?t-Q$Z6)yj(EA|DGGCyv&>cQ=X##po`qAo>C7uRx78HopfRV| z3s_O+ThW+frch=T3WUEfpEt|WSE+4_aiqzNq)lbMIy>%kGB+;MY^BPHqkHBU`}}Hk z2pZGLel@+PF8maYImiBve&t&Wx}R6f-{KYJ3V!H^%w0?DP@W{BF)P>`@+ozO3yoRG z9K$KfT2xk5)|yzNL*q8Jy}=~YPnfq{s;tA0)y@2=0XLs5`~Z#F#(afCi)vf(QrOAN zYOI&nEXUid%(~2b`)V)#1$wG$>}2?{zhWXj8<@BA-aPfpKcO+NnAfqb*jk!`Zkc_= z!j0D6WOSW+=9mFr*6#ZVjk(SoBPfaweRX9OQ%1_(R`oTSsp^@~5kkL0x`xHLihOZi z?FjTs9bqZU*Y4VcNSmCszfgdb36Ra=#p!!9l$!$UUe2%TDnw7Th52S|8!nx4qx(I^ z{8F5v?8kF`KXY9rHy!*MZA!{b=5;K~+GpT^sh4>_LoR)D@jW!=Yvy$ZJX_TM4jR+M zypH+B-b%DLskfM8OgnwoZRnoAVb7dmwHu95A2S!I<#_2Wy#1eJUMC@V=29*eZe{=0 zRP_{uT9t@F$=g)^s@{1RQKlsP8%X7M)@ z&~;jw-|Gjfn!f#OJb##DzT0~C#9yN89A%Cfx97$sEPi`8TmcCkYNt&PEjS2XlkA(X z#y;GJAG*2V7wCqbvnQ*+eFdJZPnZwSDasDKFm{+&G5Nn9`dZnJVshmGJ5GLBbqDV~ z>;NMBxW5Hi&!{!*8GT&2j8E-3%o#NCO55$8{zFEeOeWQwf!z{{~G6^2EoQ= zGdWu;T(Az6l+}@;Q44|Ye-|lSZE27|B=zB(rrD}(QSLv% zBclMwkqT`@BeX;+mHxd<3ODO?cHG$@KQr~3ADJJUpO|rLur9L~)kJ#x>y%3ixUbG& zg^a(F3m|ol)u^pb)%nKuU+~Zi{(S(2KI=N?Rf|sH-iXPlaRDfZDKn>K$42>vG?9fC z0xgW;d!$ow%mVjDJvE2|rLeZ6uU0Jk5U&xYgl@*{ZVXXW{)2JiS?ajwZ>Hd~G_|Ln z`;l51B-T$3V-j08n^sqK@42^IQl7=7)6WuYNVtRx$V5Dd%XdC@Yiq3 z4se7D)@T?^`vFh|>C2WkCw3+fe^L<37MB7`jD{g$B;iuYO-XtqXp<-$Sp2_$vE#CskLwh9tGN%{{hn?YS?|{qU@)&Pl6A=R`ZG?}kXVoOA;*r60i9pwuH>m99zGrAA?iuvCbP zgiTNGY^ISP-CP^d2N({^=CdSDDB*m{fA1?TOOiWS`zaMV2Mq@7`j-CEoUYTMAI8MazW>X9xlaSl*s zzvvKLPf{e_13aXRuPR_6+&CsF<{@P<)P@CH3bvS1_7+~U_1yV z-ti9P#8$G3LX@p5=J0`XL>6t~4#fgu1+i^h%t z?^V-AF19okg@j4hvJWh|m)llx#5poN_U9IDZ}w&oc-Eni+$!$1v|Dy^ySRmpMUKUe zzTc%BceB8SqzBO>2keJB{e;|4cb81m`#j+?_2U(?wk3<3K=DVAU>;W>QCL6VecQ88q{Ue?msc(v^>}2}NU>Y1tD&B+NhBLIsGxH98his#$;!VIDda4$W zp(~@IO!-xDQN51MjBYk8vp?IU7k;GAI95H) z`nr{C+l~Nps!1LOsqN004z|T!U?ZnaR|&y)4a1<()*pR7NhzO<7ybKCpyd#;4(TU0BweT7oWBR7qTaUg+o5(h3(s*5IZFO6 zjKun7abDxveN$^0vIY5EE`+qGLvm!HYwd%z^ulx+^t@qcCyJOm!)3>w6vOhhwMwxo zrHY8i8pC5&yLG+^{nvA?<%&!3BY>G#{zLfigm9aS7?wvg5^ANlg@szn6?d&afUlYj z?u9yvwa#KH6I>cUe3nW2h}Z{$>GdiFgRGR;>O0h~oCWR!;S69M-@+qZN6M!49mqwl z;QiQ8(35XCz3Xc~-lb++eGz(Rqe9W{qeP?R|;)U=0C^R`!{CG^{XAd*=QeM&V*~<=c*B`1pH7g zF~@xG%A)OmkA97fRpk_=41+_;m;jPd$JgBZA-c{!=5_LWE028#<7=5?%=P{~_zgW_ zeKP6KI&KCPObcjXaR;o6`NlOzx^PeG&zTsc;4|DORBmx^?{j#?ariRM=8{<-Ydv>l zhk63v?=0qnC!Ss1G8m27&;0CDx&2ey+VGXu8<6H`xze$aeop*CjDa?Rw&|wLr5m^N zxJJwP{dL@$ms|W8&%Kj*m(#e{4^MR9yJS--%*)kTZcOc9E``C)MaHK zf`}>8*&xHi$}$9iRvHA>T~~6i=RaD2u~gHI;P?AcWT-Z>p`)9pp2m2q70fC8cZ=3N zU4wt@F#8o1D@qa$LzzD&RH#K5&v};lXvKWj-mkae&o)m7PimH@y;cY*8H$av$#JIg zb{cUuGSthcYGW*|F5AME9-_+oE)2!>{(hoiy4L>zJMe7-i z7ixVrb+re*^rK7&PU6iuhcRmGI2*dypq#<*z0J&dIdJx@r3f6X%w^7$e=Xkq0#8e| ziAj|Dy;XNQ{CGN@Wj>i>Z|B1dd_Xbp`hRXKulq-Io$c&5w7IA?4Wmt&*YVyeJM~@k zTa|21$6{qSs@5utnQus%={q_AU1u(POpBsmQkKdblW^tyScnW=scbm+9Q6*~{W_T7&R@(}wR#o)s4tmc{-3s0%w3JA$1Veu zW5+xUTiwmc;~;Y)AyIEsa9k$$qDp?m2FfrH|9Qg~pK7{*x;y>VqrF{A@SS!sPwkC- zPx09mxV`=F%(&)g*GX)U-OD}}D*c-<7XCi-BYd!6+ND3i6Nh=2UFDvhytokWoej)i z%h-4EA&!0)vAG{*HAnDNKFFlBxbEMr;|NdoQIKBjj;(#CLo4VW;Rtu_?%9`kA}!W! z|D~yAL4}}%vA*8cmhIK}>uuxo)$c0ZT|dVNZ1#pQqvJSc^r*Vm*7%RUH8|-FCqHXa za78bc)A4(6V|_)FtOsfQJ}j$0Ifu+AYr~M7epD=_iqv`MaC{i{LV1v|(G#68vlY3# zI$$#{KT>v=dJky5=6rwhF#0N!80I4K4LW8~$-?>}+A+d+&fPB+=+jK_@QP64Bc=c+O zREK4Fq}r-Q_Tl#6`hRLILZ7~cxeH(7TRQH|$AGIjY$o5?y3;sasNz|u&p#{bef1vt zw!hE1%#V6{X8kAnsq@SmT4{Q)bS6QV^l|aPo=ft4D^y+s$fpZwyZMs~Zt&W#!17GE zm{wKRBjELt19EwSUW%7i(bqI_cKxcn7`^z;um{a1&)4!Z`MFG{zN`w^JY{@mX&7l7 znVq`BV3DQebu*fHpUZ`iy4hni*ZoS;OkDJpr@7+r4`8|gZlithYUxt1H7kXanWsTG2sM*%rJC2I}20dmOW!eQMd(`$x?! z#VSU#hfyWJqJfptoB<`rf;_ph7=gvwg^gIwbA+4;?sX>GEmi#Og2R(FC#kW;Bqch> z_Pe#tzPCakesy%ePE5LvXiQMPbY3W{yUt_CYu{)JfgZhQabd8Sr11=gv_4ZBgGQZJ zemNWDC@_OmZ5p(>!%LiI8Q?r?dd8C%Dp8U;v9Y>Kem5-Y77t+2+SSBeU%G2Fe$f%T zh`7LxYybZ%?thhy|MXqRW$dQRpKYz;de5HA$A3GH<%p&IBe-+av~B0jwVdX!ZuIw8 zM>|YSmkzAa{#AdtV|e_^G^46z7x7o)A?p|K9fp3pOcdO;5K^~E+ofLXBdgxngSMy5 zhveC&IVQaYhO~p5cc+8b$bqwrfsUO&gbBFQ!ifKy3B)I(2peNyFx_>d4wxyoZ=K|= zmDe*6emz4klu!F?6hx3aCv6eOC#``H1v{l2g0=2Yjn3JPKEjcSiHsKcnv%amn?P=8 zfo%2ATeQ^b4+sR3E8^Xx>y?KcLIU=F973|e-RkjMw6~!hA2dL1|*Qj zXLw_3;HGUv4>DHHgJjjr+Fy0-y9tm<_{kY(1ihS6gP(>EeT8brBJ*N%z}#SN)Pc73 zRzv#W1K(6a@2e-IBTz0RO|?(8uF-*psBPWIH~hF&sX?~pXT)oIg$!dO2nau;U& z0{!mC@+cU*J^_OqySTGlH%B7Dr7CGpafOKYzdo)}uU%*7OF6K|z}|-sP=fwNa|p8T zu6@Q68~ND&LmF$*U7>uY-xK|PF)k+6*)E~H+@FH~C_mye%v=t7hk30fF@6(NAu|IK zo|(0DJ92dopgFWXQZ6LoRFEi|l}uLV;!={?*qG#Rp6u8% z06jo@G;f(^TW8bfEz}VoZ`3$%YyiPg1Dxyuv%-a$UDPHwK0b==X%Hs zS?t)2*WiF?s^2QjGt)`YeSXv#Ly3 zI&&Y7_m>HfLr`jCU`Z_V#;!oyYuHm{O#;0lbsX*p?*fw*$T>n2w2&v8Has|KYe8j& zARkJQAGmbLL@inNzbEs$g}>H|RX0ACGr{#RnLZ7uLuLe|VeRFWwr1NZ8;RE}QaPna zCDfO*z%@VVj@AW8=aj;Z`Dzzl#7(k`VM^~6(>CFTbW@^C3U!*>G1x<Aa@VM4G7+|j%*Pm!ng~C{>6F)oprqWF8ccWp*0{eUK z#}dq>!L{Qcb2mTfop)pZm8C=Q)lJ3`;CRT?c``7bb?(r*18ru_pT#48yKv&XIae*3 zKa0buS5gN}tM!>Qe-<^|TrR{o;uNU^FPBt#FxbB}zr+Uh)Jo5Rq@5nNrUThVDU8v( zk^GWBfJ;q9u67%}=0d)N}&_Q152 zIOI!U51S@72fn_Gi-!N3tzomC-?rk=U!#i+XJ71yFn8XA^|<9YTHjBhRa==1pIU`0 zQG61PMMw76ccoA0eyAa|z_6=%aW;(a&D@GS6(=99zB3z;3N~3s>() z8};Y%2pCLqrS&QwRIeoi5{eR!N_*scHe_K$F{t{o7Om(BJ~N9fb-g(L^2%BIf|0c- z$f-p^;?T)pv_L6kpcv5f+sPNsV2gtR%p*)H8l_aJMC*DeBuDu}*JCC91g&=Fb3q$*S3`UGBf?}ZAP+AaQYEV%G;*1~Imq@a| z2~7(h3{YlA3~n7+)(eh_SL|n@8_vNOa31u=*V*7E?E9kuz;AD^@^gd{ld-3-<%`4s zDgy=tc0A!y4xGlI)1ok7X1{zu#$=q=(vj{U5w(twPZ?#7aP!V};GZc=tb#l%n=oLy@lMEyFkS%Rzhg?#i%goigQE2xy!SJ zyKK8+3)*^YFRip^wjpWdF8BQIstIUE9W0N8fxpm1B-EL`sG7TleLQNpc#{=6_{suT ztH7>P<3~(qt)J*pY`SwegksNIOO%vzN)XpI2LaTW z>tQa;gZZG_Mu$*t@zOl0{%E6si)7j0UdHbi4hRPYqE7Ko4<7C^6GF$I!1p39&i{7| zMa%rLw-EfGH*D&>*}TQPm5B7}tU)yYRiV$vuG*sWu{&F!qUTgO`FCMFqJCj6PCT0{ z%>x??8DtOjh;&q%n!H}x5QZTjpH-dpu(3oDb*DLyw8DlHw6IH8_?>HmSB<@vHg+@Z zcsi3K=w&g6S{^^&j6<_jIRjjG>7qQv2^$Fm89NT{2ly3E!Ab@St3|0ZI?972!itaR zqUbJ`!|aOcj+z{h7u#zjIu2g?{UCNhjLiOT&BtvgY<;s!=USKz?tOx8_W#5D{eBan z^pOY2Q$GFGuOC@xOMlDw=5oUSpYKbP5Ove2XthEvZdEirc0C2xqIgvVe^WSlAXv9r zumpkk{Qiq;iv+@BkS)s95FxNcPTU3I_thXRfu*nvnqWD|)=gcaTBeu@ zDNh~WT-x$kXNgL@^tbcFS<1Cx^N0bV9bs-tx0BdB>1;V0+)=rZ-JU&t{;fK&a4~y}lsP)s_e&gL4GyLC5o_!U_o}4TNo$hGz@fXDQ+*2Vv48KsQ+T7* zw+4WyM%3c0w~lfFeEpB3)nQF2j?O4b%BjFGrLpe3z^Rp}p%x5lBh|QxsDLg_7C`DZ z@B-Ktx>QKg*Vx#?PZg;{)F=7s&<)zc5=J8-JFu*OVPz5nGDei%c^7r1-6P1 zN&<{nsy@3?HHGZ^7{SY^b&Yl|HIn)^dI!XU3SV9n#Px&G{dLW(uNsAxDg@$h;pE|To<3H-O`&A{xIG-t26?pv zSH!{^N^GGlU05+Jrf?lC2%HYKHjhRIPn%U|gX@i*nzPou3wU)lvb7EHp>X?{ycmwl zdp+$QoleGB@Sbw@598Fz7HBo{XjLbQyg+5r>D1vupBvdAhlM!0ZTrbiIn~1Og7dyD zvgJo3t{GOP46Q+n{0`)XqSlBAZ)_cQy1c;baZ<13nBu&4x-A#qQ5ebi*zv^i)UhDp zhHGKMb$F9dXP#jWrBi;D4`3kGXrKI)NI@v#@n44(qAX4Pv^`VsX_{~7DlMH)E82;hdUivl%ECQu9}+;^ z3~(M2ZzVi+?YH9~+oa;`(<_gJL5svbjml~>qo%RzXPp&TAv|<`yZlm#a7!d}WL>o~ z9c=b;F`82AJM{Wszpv3ubq2#9WT8_nMoCk@T6-2BKfPlLj)q1oZjs&n*88e*^r z{{((WEAV_{eqlBW9QvsWQL-`bLf%b2PK^l5wm|_T@f6_YY)G8)>96jaDV=cfw>>L9 z#b;zcTOnnQo&l~eO*c%`cBv^8je#L660o;;T-q~*FyN}~n(ex6xoL%IrK#Dp%0z$~ zsV2p67<9v|t(j^YfjYI72T2Unp!M5*Tx~g8D4m>~3yB068J~6Mn)yy#)SMzuj90nR z@|0Igym&{>0#~%XvNNT~9tj_+6u4D}=tm#NV;M|+Xsu?FHjh_$rQEA&W}q4)N(qbk zFQ+aenJ_~`X)B52Aj+{C#mPLw0)oiBzVI`&yZkUhvzRSuX%@@5=7&=-Oecm;iL>_# z#*Hhv@&w4yu2Z8=Km|raz`oC;Rld?1Hs(RnH783bGB|^4sYj7uK!~0lgkBP^qrXY+ zdC)H6SZjFfy1O0>+=sfG0eX=~dWXw{l-pwTB+xEQD#F&4^k{9lAz*K}@3W%=K)S+h zo9QEfR0F~K*>9o&2Yu4#uzd+EhfFFCEG)Q)t_h8$z5M1t1JOIrfP~#HDkk1Gr~}Qu z+R*GX;6}_&nBMz5hp4U$rWyj&te6P7v1b!fxugED08;5K7Gqf`CHDUn zQXf9o25%ytzt6nitkG1kwPE0vJ)Fw_hVPpMNh>Kc`xD6wfK& zxNaaURF`5r#h=E;HVKdFi!oI6f5y75r{nrIJk${iP;_KSsmNZ`q7s0YAr!_&s-9fE z&u76!>0EZ;sl)3H0s8}Ck-S(A$dRHk-965PglLHZE%!CT!sj^44Oev!cv_$z&Qum7 z?y-!xyL=X|*4tz+B!p7cC+??ily>P9Y}vKOB7SlAwRh0928Ghz*t-x)D0CW9Ss-7L zbl`0&%6=aTE}3IXtHyAY{w|D;p{r_i5iJ5Fi>3Z~_}9H2%0J)#UTRbJMAj z$R_)l8nO0lkfSZygC3k0`%;zCDY&u|!yRPjAm38lJF&{Y1H(J|B>&^&Qc#IV4A=t2 zmW~nGn%bWfnTU$p_h1x8k;L1Iq?vsY42;HX>jXqjv6?Amv)j;O1u`e;@I>n*^g?7U zA8D<%@(c!5r1tqzRa+=JfzG0TVw`MfX|w6!h8_<0A#x4kJ(_Ah5(eqXtON;I$EO=d5uG*sQ?yJn&;wiW`L$&{xc+R1FXyN3KV9XUE zC}^zy;y~gb?fT=EQpJsy$WR<_XfZQ%!!;Y+hdj${RN;rN2pT|SHi=5*WIUyrGzY3~ zIK7cOy75K|j(sD&4l4Ky=P%To$=6b)p9((v;poDlyurppX0+pY#IrxaC=6dO)Erd^ z3TZy1TrrcGB5r%1Z+1M#3q7!!E;qUEV3*S=ms~%! zsAJ-Q*;!Vb{qv{b>5vWX9+!^iT|Z^9Qe@XO$euZ$JMnH|WCLV`Mxj~Tg85p}(xZy50o*b1$1H>Jnk*8r4!id2ril7vmZqFqyBr&^sH zMA3?BlE;K@IX1$*SmBjoh3Ik(g`{e(hD)M5hSKU!6uhN|@j`NR_c`XNOPG-mIV@<_J~HL`=v_+c-4md`^ue5u`j+{1$4l&eyp7)c~d)@w-2kUE@w7$sf%dk8pg znl)}~KG9b{1uw1qFm#6LUBq|!ae`yy*{CgpYE}NO6-sukPsJ5N1D=?* zSP{?p0$8od(D1aoprs-}9Owf<>h!Ejj8uc^?m`a8RSb4)aJX-YN{DPnLQ&M?AELx?oA^HU3sX+^QmbS2Qp<8q7_oDYQM^47%G z#6E10o`xzHTXgx3PLAKO*6>`coO_kWVPX!*$DQkKYwX%GTd`ulhf7YWa=I4VTplo~ zOR31^LrRnisS6DFMfY%E5O(Qn zwdqe#{H-bBvqDIHmiUm@^46n#lAQxf&tNG+r~CsLd(!jvhJ(7}#h}oVwztzuR6{9w zG;IUNa-}JgK+)yLfUHo3E97>l9Yme78iaUMhSA{ zI`2t*>R6DlFyT7XCq?^Tbc{cy&v7=2bYafob1qMn)u$q|Uy5$UK@oQTOSx+;AJxI(AvIrFl_ro`omr#vlY)dmz)Mtmj@$JHF10%q@4B4w?yRb zE}AZxE}JlwOT(cW0**~$w)uC_+4n248EQZtg4WNIr#A(X5*+Pk^jtVjP88 zT1zMoQr5e#yI53QO6t-;oJACyJ2DA)eijpzriW?JFSsuXq$8A=HO=vYOl0Z`1AuP_hh((s_@r#E=f~+s_eT-#YK0xx6|cYLF58B}T)~x~_{;%TVH|a`IU4hBq~z zQ>XNWzt9oeS3O@6E%q_seFo2gE~#l#?i!0yF&TGT#>J(uKqI7c*pMs+=l6nV1FNj4 zW6ca{ijV8)9{@79)_&@K%RmZPPn3UZ1x%Om}lY%NpLpDRIus9hw@HsQU(P`Pdxq(w?y-EyQdcGj+q zv+PyqW~5Niyxv?TF>)IJWCb@S!t(GsNl@wf?*(w#s*{sP^&PWeD|+@ZOC^1%BeEYc zWoVh2!2qBA<`hK#x!v6UaEX972L=h|jAhL!47w=@-MUUT%zXaUbdDgH4Avm7rNhFS z#x^2-9|HE>um|vf1D~izi=8;_Vn~yTQcL^4pr3F?IYC|4=ry|eIYC9+55HlKgcGY0 z(Vc$hW#v|$I5viW{kNZ!msykl{9%V9_T%!~?p(r;``64d)~3=r4Ah$-XF)=gR)H)U zxuOTyLPd?R`sqO;cj4||q5R}C1H*T2%ZagQPzGxvyz$oE_|l5Q9^He}e5k&UgeO8x z!OP;?#G46!k<#=z^JUK!qL)v}Cw>ScA{sF98#}J)5_jTkBwWb6Z`W^ky5y%XXQ2S9 z){c{W?{ehG8RX@BNF~Gdh^Zx;d^DKO$+bPXApoyTh&l-8PbEL;oVAAN7}FvK)(g$< z1-<}wWE#~kp^iRW#()<~JQ*RED8~CxBQO2SP<|8FU_vY1z1ytgxl$bFVK6ZA;V;@R z*)Q9J9&I&%%29HddD%1W&5VbF)<4n!;3VC$`)1J# zop;t2W6*DEDCB1jk_|Au9O?wO%PIa+wKBPn&WOHp=ktez(|4!TpVX0a?w4&UsiB5Vv zv`B?kTs#NlRz?ApmLnbl_7~i@+)J*^Qf`UJ5^KvGndBjfPEu{KGy(?FT{;D3NvO_A z^I)^Vh04pQhE}X{!O{TnDh;7IK2!11DR{D>otCaU=$2kK$cJrPCAy$R?P%j6bFpm} z%4qcCt~lvl~}Jn8E`k6XpsSFHCMT#MIg}a{p8{F9+(Sqw^`4=o4=h?OVBwU=4YmD zapLAUt+Ya`?`DB(mwg@I!jtq_TCz1x;hk>q)#F>X%UK|wwLKG`i%ZF^N1Rx_485%% zg=P$6kYyiN(Ey@AUB5gT$7eZ^xYD7oLeP5S=~Tg76au%<-lf=B%=AmmoKlZIz9Ptj z!1_y^qs{)IQjJZ!1UVOyBS+zB)6@jA_NOb^!sAsyS9?DoILZ*t<9~?j6W!6E6vu1f%|_*4GfiG-;vO`I_gtM{}b)x>8z7 zRf&X~u_X4m1-pk6(kM+F%;Yefmh@i#fAfD0-yanc7%2umdycb#FhHP47(Qa8b`aH) z%>sAmQq&-RXEC~xk`y{L{WgRi>|5_qNUP>At+mQ`%l!d~EI`Dy|%jMx}Peku%sb!hv9=C+rjP$+#ZAm0W zh7`Eb_Ep}6TpYW{=27eDvIbwPb87*impU#keNWn{G8^2_Ec9k-EmnjUj7*yndW%F& zM<1Js<`*k}M(F-h+b1kB=?xjuX6^efadxxoj5pewaGkL!%%gtn?nB4^+<+E42GYOu#B{K^_aO{x|pLe4Rm7LUc!DnZngBK&_vkpjY3# z^})1VJt9V>c){6gp|}iMk&-^KRyO9p!aV!2O2qM5kT5qXOfJU2t!J%#eJOe?5$hb< zEUwb>?8$Nz)K3Y82Y1OnifG;O<^MpR)gq^Yc)@nVBUyEQ9m)r-ou7Y0Au#Stwb7UB zxyqh4J1(Ys^@gSw9ctwi+(cozmDAFelx-rFc#Vgw3;ac`pf-yZLcuw|4`U)(3hB8<4ouhimTFDy*G(y~Tu z)_3+S2L%-)o2|IVhVY>TbNKS#2Cybp*aVx~9f$^#?c=%PcBf)U8cA zK0v8d2q`m4#G2N{{~s{!VzS>#FahHHqBIf^;Zav{t2vBuzGvH1iJbs*LrV*tDB;Lk zj~okSJ6EkNyaUCIPY2sg0z6?YRqQG@HTTAwj?6VbDqn8GIrOA>+2>NR(7k>MVGL$i zX`HH>m8q&3f2~Y#-Lq1-2D(kdpn}lzCl690tf_8^t`|{&jCUaSD>*(-${OW6&wWVe z%q35R>~4|9(=zpFj}1^}DlU|Ve*n2(nYHnuk*WN;Iw3GaHHX8G=);YLj9Z=tu1=gG z&J<^fY^h_q*vsrB(0S4aP*`D&q$qK@3sIZAfkoS=Z`4l-NJk!|JWFB%IqB>5`Qr8~ z4_NbO%2b6o-!sA03Y$nm$~Uj;uZk(ry{U>tX$E?%4J12TepsmLj!S zC7)S$Bt`6Nv}$7beQg`4%))1@CG@`Tcv!qfYmJhimZ=uBZ)bu_(>`UFi3dcWgiH|5 zdpl;H!A&=a7M1&uN=cMvoW&=UwM@Y;FJF(_-{jFR$Cte_N|s!o?zMF zUTSLr>M{#9^1axTEv#5Ty4;|m7^Oy52%fwAa}zd-ATIf&(%g0-kbsLyAH07?l*n&grHD$EBI3AY0%ZS!ZpRuR zc{Iqfp^!-Q^6<(!IhSiKM}_}hmfg6uO5yw+_$I_qF50)LvkPC+e0xnzxkXbOW|h87 zjc0#OG{VP2Hua6zmlr=iFxQK7wsDX_DqgYw#yUs&U(fxheGtxYw9VKkB#&a*uq+RB zL>!{j(Uuh5yS3VKR5d!kZE=7n^Fd18T=T`Hng}Kp13=fI+Y1*YX=# zyUF0Ak`jQ+Gg29qjwV)I!=xLTZ5bdNh7>1(Q*7;+i*E7_B@waC_7hH%aW2z8Km{z-E#LzE1Z1Po4 z#@Tl^$e}%(E`a-73eo@zM-Z_L4*3@+aalgOkBj_qlZ7(ywyffOSW!FmEwWrX?I?s) zvU8{b59%Dw`ttc%c2=REI^uF6k(7f+Ync{6hy#UwVPl$lO3A>L`@R|RGnT03$2 zQWIa6;#hhjMJ?<0PGLr8(O?{JjE406vC&adv8(K4fH+>_(VN{@&9!%yB;h~mr&C2S zGAT{o10z}xI;th?3#PL`DpSmzB|*$P3q!wgm9OVM*D^Ig=zmA3d)G`fb#K<^H80x$ zx72e$#?CzWY+m!px#9q!%Q4gl+e^!dMA}FFRVZkcVM>i$Dl0V~RdmhSA+d+7wo-kq{1WxC_FU4Md+{y0>vkyh)smMT)fuVDDj=>De{ zZ3{ZOekkluqXtwPB|^kUxO@KuSZ(q<4-B4?O+y zwolCEg4obAz{z7^$o~FJb7|S0!WhMP9pYsiWbI|9S5#yAlahM$aYxBxz)P+H=AT$9 z+|SWA+Z%oqoy8^fC%Q+;0@vUF!zD?(YZmb}4SP9U5E%h!eSY6;J4)KGZaAcAOFP=Y zdHC*I$2bC3kpT%LE;@WhP@rhW8KJ8^j_#kSD4JJxW)W`NsZl3k>E_=Sox!=bCA95s zbWnB!^uhHG48@^hh35a$1&g}uj}S7~O7e8^N~ss0SlQs3IB5^B@+A6H2kM*@-qzwN z+S85X51?R^N8bQ3_SSQBVmP)p4)5f3j_T(sfxOKEH`33*gdnN|nFdm;uub^FeBP`B z8E5HBC0+b&-}UJ{!JNndSHwt^7$#1x40v&8PS=I6@-7MZMSQ12!12pJ`eR}Q!&7!e zf>w@@_0Fa#6VCa9BHG;Eg}jHJRbsIzj^+{F`7MOhXbP?C7FjD5OK({K+Ye=dYlhHc z(!2zzEU)A|_V%!c_Tf9|6?FequqZ&N>5PX=(%sSM3BPR~+ggnPYoiP^metao7r5&= zK^_a<$0>RZ$CV;^@y@w|>Q}HL?m&=wJyFqv%gqC>`+_E-hE0F725#EM$v6qkpa~f!VM+=qoQW^$nWS!?KT!ZWI%yG|# zEYwIYZi-avk&fsaTcLczZD5&7ul;u+pXK($1_@BO(+qHyn5p6~MJ+Cj<=(o}Ua}|1 zX{ubtgK0ruk2Kv9e)w&Jd31zXFW4U^5a*lqi)KzKRfz2~3tX>lZ*1D? zRZ|@o#F+5U<4Tm;H%h+SN-}N-jnr8uYN-agc*IcfhhCb7REM&UeQWRn7ei-Fr8Q!P zOxO4*k@69G3CS`yAt5OppUXW*bM51|sFmzA`yOJyvGG*5bcrX3a*7~NgN0~VbS@;- zSy_D-s&n}s9XB~9Xe>|$)Hw_P}gWq@}oXTa_F}nd$b=@uObm+`=Rl4-+{szR)mu1cXH|abz{K0K$`SUlU4$c z&TzGzBz9NT^2yVg>qtNS;V>u)>};_yboVJ}CliQi@n1h-LEow(6a{3nQy98Kq|IT5 zLedi?__Dx)5&#`LJrkTyJz9QMWsp-B2h6w#IwH2EjdYyb8^EFlAIl?P@JuuHcB%Tk z21hm)cPeA#Pt@y+YOcok3v1gGOV8C{B0dtlVBcUR_nyW4rLS)A1lX%TkFdv+YEd3i z%gG~QaD=IfRg9>ZpRyeJ0P?@`oJ+XtjFfm{BZ>6R#X}(}syz6$XPpCSM(FI7b0A3z zI*LMqwAwmeaBLDwZ3JSCrg)77Fa3mijHJ)D@MX0Xm%Ii6oOaxFV7*FBP>>GbcNRJK zJ0dOh5|vjBiLSjDc4U$*ur8CB*%vL+_RC1YA-YZ#iUH1NLw?BY_2z9$jxZ6Kg zz*?%fYB4S|Y@E8oMXG8@-D26!07vG|uAL?tov~l+T76x>t2^eK)Y2=4N5EZu-C_zc zCGFsvzPe{7bOB>IV`>hoaga%I*y9nk>PIt#0u&F0M8c#I1I={X6$3B2ILp5kx7ZCBD7Jkg?4C_#zA0h zmrl4l-JvjFXMd9<%`{26o;!=5v{888;wNN{(mnQQTX&4s7TGr3fwhQT-m)6l#U0GfUeO&aK1T+ktX6xQTgz{C*s(zUUyJJ$aCF zJBc|T^aX*G&^CBOx+&d~ZcBFvM*v;nHbr{wk(S#SvRtyyra$9$3y)Xe&+;QGtD{5^ zvLe!vp~!SU9Dv9|{MtOPq({S4ao=?6p_^V6HQf|J9tNrOsf*WHMLnl9p)PWS?y=O< z=r;Hqqp*r{ATg?_P}QGP`r0t~x>zis~ZWC}rt9gW`u%h?mI zF5?zn+P$6AxCAC?4De8UY_@6ht{y`7r~o`KJY*v+hM_?wxFX6^#~mjeeclwb`q}W9 zr=t#D8;V~YG((XV`HaUi;;Xq*+YL1dM{0~(LR$N>JKsfRfK-_(3hMSma`XN_L_j^~FiLW*eGWNLo8XJa8L8<32TB-sB zq9+IBQ2s(!&^IPGKEc*S1LZL=B$Up_0^UfaPrt-8$6HAkag4h|q*l9{yphy)_U^EG` zBXI(K|1@uZj_)vct2Ovz^aNwUd!xT5u2hmzb`Hm);as=h&acirYZ56g5c4h1SN^7@ zNOhnfY&Z*!5S6N$!TCqE6x0&ilew7 z_cjEs5@pzG1!*8l`UYU&57Yo&q-J(?j4!VtXv)*W5IT|`(rSvKNje6+R~e=(S~}lD z(Kl-fE=6a7E8+=L6ud zVrOz19ZMJHz7Jzg2)DUTA?op5E9tqUj7p5wPbUo2zHfS9dT8o3J+$_jV^BctaggET zhWEEaU;F9a9oVkGfbdHyPqYNDP|Nk}fjc58p4og3}R@y}Dq9 zW(;RDm5OMH*hI*sJM%^khS~x}AM*HQ*fyK?gQ5s{w8DNUfK+f#bh7qQ*C*G`!2HDI z(4Mp?VJ)NnPXlj9!QeJ2=IPMh%;wgX9cQew{S5Td3(2Vr_Wzaz-6X>Bwyx@4WQ&d z+|lUM8wtCy9ZH}Wcuh&?JBkSB@K|QJuXk7AMq*h^w1#Qs@;w*^`8TnK3Y+fE#HI3| z#~k{q_Z9|rd_SfeZ_`Pf5>E)N?Bk{rIuq-9u5FqZrxSTbjs1(_;f>EEBF~NSnuMAI zttOOh?TR_pyy?a_W;AwA%vUOP-aC(kR@HxvWgTpK^5T|=E_*&q*kS(ahtz!`U_;XV5k|o27gpowV%V zghwTZF4gLxO^1pUQ!QpTj|xrT{i3e0Of`~+8fc*EelinWFFlmD9DODcKDDnrUkiO2 zCsY+DM&;goYtwqn^8cGyLxtMuvoV75=dp(Jt>p{wl>I2yP)#bFKvN zChV5S_K(fB?tCIPoDI0~(U}ggTgFGY&f%;d-N**V@%p7^j2C5#pfwX_34H~ zS~NFd-O`n2N3aM5^`k6XS%qRg36+U4 zkPO~E@F<)2FveYIQc`*9P_Y|qYZ;U70&E*J4(HqGbf=wLALltDM%?m3Ck-+ zGQ&xMARq7A9${);*T!M_6n!RI;mBefEDwuP`OsjyAhVA*IP zn^F|DgweJdJbTOeT$IH$bJ~(e=l{ptn?Oa8rD?+_B65j*Gf5z2Nfi~@#SQmzscNda zs;hdP>Yl2uUT5s;WoCM&yQ{lrrf24xZzhril6_MY6eUsIZ~?&uwbTGbamNM4mBbZy z+_yshlbK}P8#iubrvLZ--~XLD$L@2u5gBpg-uJ%C^FEKsF<{5Mg)tE2KI~`*ov?Oh zNr@PVryKf1rPjRhojz%@y*YtB=J_E$o(8hYMiC4~zgzfEnRgJ2 z1MiMNWH~t!QvMbscvu|+5w0ndBJaY4Gm&=d&M=7xFwi~txorF?%E`XSg)rf>qscbEJ(`c z99#|;uqtObQOWip#v~_Wc`NvaIU;FE*_hFb7gbM^HfEVefhE%_TXBY5(csE-X)w|; z#ch&ZMb}a3`aIPJ3|+csO`J5Qp5B7S*qqn5qGR{>^bq!-<@M&HMzmq1PL`(e+1F8+ zVxsqM<;8YgK<8&9jk9&t_cE`b1iOK&*BoxrUXzY;aOvw4qIdKy5MdT>8VM9+a<-;5|sD0fHWXIPlq4 zr^TZ{uBaMOS}^AH_8dp_*gv85aul=9-bF2_0=kg8WhP|qZTwgo(@h=o?w;1yKas{X z5=ZOu_*-qq!;uwDhKOpOV(%2HCDeVF8M5u~Du9GAmNtu`BO)=hKSC7}hY+h@+P(mfD4j%*T)EZ5(K>`>M01m79+Y+!dJinhi; zRIar{dt7@$OFl8#rkdze`#Pz)ugCbpm8(^`5Rd-KInRZ02oP&*estawln1qVIJ{_E zL%&X&v3M>THwL0M80ho8N|@SFB%%)9STJ`O!EH2vZsqGK*fgrz`KTU4Qy z28mMkRN{2k>!*nhCl($BNeNBn)@MQTc|&ly4FxY_GDOM(Iuxxc$iTw$!ydpxc!Vo9 z*#*{_AtFVCNig!X#{Xs}8$^QG|0_r3iz_f8#gX!jcx}kQ=`$m+@<}3Cimb}rRW5we zks*PbJRqE6t6CTT0wGGs7mQeSd?Zbd%}*FhwJE}9^YU5^Xj)~5?J3>;!dqI|@mOLd z(Lq}}^&L4bE#6lxKiU>X_vO4EhIvv-Wafab`}HosW{l#(uKCs3Jx{RG)mxD0?Itc5F!=!*#QPE8`$Z{+b zp+`y1Ouk$|XMe2QFt6l~TUPVPP7&a4pSZD4N|q{aObV!`C>eqVZ?6lhvyH}@$WkG? zTPtH^M!06OqbUWV4zunBpww|96jf$G5{{}r#2@C@hWPOKa>gP#Zj8Ft>nG$DUp1hT zI&pqxeJsouF+(s8hFy(v^4)qm1qV?I7z0u3qtpmEBE$?qi^WPYrO}nlB4#mD#Vk>x za}`$XLzo))h^*EtZ?sMLle0*Sk$!nJm>bO9rqgf+y5KCF0}G!YwcXXKMJp)D z437lTa)_MZKy8j5zN4;)6MCwaI-e7lpHqXOJ!e3?1hw`DEjyFXh$T|;&c)LG_C?bQ zkFN}oE&mgmZCtj&Vekzh5)=xnv};t|nq8x?eE3+fUX4=eA~h5WQ#))T+R!r~k;+7> zZ1G>j35bjo+(*Gt%`uHn^3S(f>pyjsNo&vLNle-~B}Xq`mlcPB0;UJa(W_s}dd&Y9a9h(D1AV6QP(2@r`dzRW%hxzhK{*DH&%vg0gUWX%~{Xp{Y11 znrD(BqBrDbh)qw{<-6((rvw*rC8kjO=}CqMb7r#(%X9`|sk=HhMtTOh8)hcx%i5_$RI060%({~2dJ?lKAm0S3&yzC3<=NSY?I;z%snP;zPnm7v0 zCc!c}h_2kiZe__b*rWw(-^G4;Vn7aDBD4G^ByI#aXBx!Qjq*glHrdZ2CU`wlA$pnd zu9h}Ky!{Kofy^2EW^6Yk6lpf3To3xy43-0O{`-I#4RfQw(y4v!1^hG6h|iO$wmXENL*}r0J9?7j_zCgiClIoV*Wy zGK^Zm`9Hu6J;E=<1~E-~z(Eoh-%_K0dYefon2YS!`!IQ#VY#7#k^6)JT8>GSLOM*m z&fVbfdj#rQR%YK=5iN?{KAO&i?dF*fjF+FLBb_In2D4##Z7~FkF;u_tCH`Weit)-L9J1J z-dEqmP)9$64{%Vr{}9UGtLxW(S|QDTn&n-X=n_Ns_(msM2Dhn>)>LFfXGQ`SvFYU)icV@hYr6|4z6aM(8 zg^Ze+4gC0XV63WzrClv|#%K%{OSNlv+A)%rbeKR-~js^Neq8s`5KI; z^e*~aMTDF6b(oyX<#CjLaKG;&N7sD_lO#dIpzmSycJj)k4ZjIfq#pA(xRq%TGwo>u zADo%@VKVAFe}n0fyZ9f$WO^z%&}!F|xL3P-!wzoOZ@{Ex^B`^}_&M@Ej>jSnKG8Q| zT7Y$<_RZw7=pVq81^PFCHPRRTI!rnjLVC0TAB9j+m<|cWp(UYrVVl{%BTSgUX%OEZ zjPq-*?D|JcrZ^oYD4*HCOR0R|;9N=~p!*4Y6s$E392F4!O*0^&KbUAjZ>jDqVX#)4 z_aKct>;m6}z!?Pn<3pYuEM~B8@fIYVW`m1@h>t%N;$??faL(vQFnzhUR=dK_Arj=u z{{~E>6KMoo{qh9FhcGRG@l?Cd!339t_jfTt?*`4P@&xSjzXRiy&e9vWdXpel9u?3o zk@xro6X0#+gp3C6V{f93<2 z9B5_IJOYwR=*?VyB2<1CCJw@6gLef8QH1wl@+tlQhI4lk{d4<0c9%3U4Vf;+(|hdu z&=9uSG*6YN<*vm^P^Oj z#dsLDTBqI^S0aQ7IRcO68w1Ll@Xs}y>S{S;Xy1Xc?S^tr30eHF zlv-&F4w-^RKw!6YOFURydWF&paZ)5f+HP ziYIuLF0+_yhPI@6JD<}oV%(`gO3{)Kr(}o_jQy~cC)R;N(QgXGF0uw{z^oF�tSI zdEu1SbYHpTcimT(GuSqssYFN|;pL2lgO?iCcQSpb%WJMB2_h8DL0)jQT$gMjjG!_Z zp-CkeTT}2V+Y{N=BRbVbf~kK923jHQ!E{Kt$KGfA*asNIfO21w|2-DWwNZ<$<>3p$ z7lv1aSB5LHmPrsPYebV?MmKh0LoBLcCPGB3miCKs1bMPbwS#Y(>Z*`>WkszeS$%Hg zvY2Tw;<%nfSIC;xk6KPF+oQ!XqYA|CU;2CZ*C531dMd2<-;Q@X^3b4pR8)3yZcqIO zQf>MQwSs+~xzV*_1;T)@rH<-n$_`ePN~454>hFzTyK}?O5!*aVgQfqd`;_o~X)pW> ze=b2Hx&xv+h;BMBwfJrppA$BL)IHqF zpJzJ_(&`)!!!{Vyhi~dMwcA}9ytK7LGv411kjI8}%Fu_Cvivh?v_kYtb41>m>C|AC zCi;Zf0>Jc^HE|tzQnQPbaqnU;P7Gnki3+rGQSgUHeRVZYf(A`fA#WOTVohlXIuRA< zAO(a|0oc!;D^v&=3}XTe?~5uij8}uMf7v;q*6u=w=vWwbWyo!PhxWKu2niGoM)OTk z{H7=izfN=W&~*+WGN(bDf_RUneE+eK<;2RuiazXBtizteVHNW+5dEVPW2v#sLkK0O zM(y`G%m;4^09 z7N;Jep;91fp9(ny0Uz*l^bml>7~l_R2T6$ywM|UJDhX2J?a#Pt#?h1|A9kP_bh2*O zhJxc#^cRP%V3%v;y_Gp!*(rpvOm(}TN(p|&7n=YvpZQ}=Cjmw&5V1o;lf&7lsat=U z$0i~XEd2%rno5BMSCQQ%B{!q_{1Emp{OKhTqJ;{$q*@P0YEVi5!EGv1=DAcK9A!Ip-RMd|5>)3kB>xc!{6 zo%6Eh%Ps`goo-2_m?Xn0O>Wh*O(^tFwWPtwC04)0ha{iY*I-;hRAH1-LQXBd&I>s; zzlTML*ohG1_P1*E4^(=N%7#2FtkvP$7Vl2`z_!YXF)ZS(Ylhl2%wAiK0d3Wh0ND}f z{dachv!3>hL{loewL^R3#5E4NWn)OX?I&ER*4N}Ox1C8aq=ql$y@Q@|QmX*Tm6-?@ zya`ZkyT(JJmhQqz-LHEo!)P?^ra&+?}th!tYGtPvvg4X znWQ<9lWOVKZ~8B3tfxart}2Gya8UwUc~v;nX|?L^q1TzHLT#?!aK(;B#ktLE^^oNe z+kIIK{@Nd%83LxxFbYgwoR6hF6jccfCTgMdEG zg5+(c?WP^3ATs?KY@J}kbYv*5{1NT)b$eSRYAfOIXraZ(>UlZSrO@I!H75GU)m0mj z#F2ILei^|EKdUyA#iPF0lQKLAKeb~2X9VgvsE%n#Jc2#zz zNOx*uX|tHGIrXw}6)NZd9aZ6OVb8)k^gY&79P&j`S(j@uk1`ELlo|X6MRvj5jM9l= z{#8`(9@aKwN{q=LTjoIO68;?pGjEV(s`ntRCz5tbzfq}cbOrC`joE} zzNqagA~UIFY2W)cySk;m7Jcve)bE|T!*HZTQqs{Fo zl~6|gxo`JeJN!Fo%y#M~J;>g9P_h9XKGZaUoqbrk_P5d!oNAdIgtQn*X_f*})z+(e zT8D9Ca))#<(OCq z(&rR2sPh~9xZ>nH(wLVF-E$Q+j)7&_$msr=KJ2Kb-dMLQZMq|k@$W0w?HuS0kHI(^tH|M1^h=| zRNC4Z)|Dx>$fj8)z%V(@w5(>t=QBM|5q(`bW0?RWR&a07(UAa_<<|Q;qS#6@qr3%) z7Yr)+C;9Vd2~k~0HX@8xdG&euU5sO8I@-f2YPkn~n=xZK%DI>=QC=xD{^mHb6m@lI z-RsbspZ0ZI+TRk2T$s$yTac)N-y>^#jztsuYQ+)KG);n$3WH0;OKqLuee%77h9dyXUmt z&bCNHll%=0X>Zzv+#jpIp(7TbT!qf@A!IJ$iA>nbu;bc?7-TC|b^Y@PnHMjwFmkd; zT2ifIZna=Hp=LmDe;1hp^n4@&ls+2Fj>uy&70!Gi%9SCart5K@#1mA;KS7A(;)B2}u{U zK@sE`T<4=#*SaF4DmbOMv*)jRzB&>uLzxl%FUp-yG&vBi_A=+zW#PSw zl)&Y)!}~5TMuE#U>Y#rbSYaJZ=euV{~SfVCZ1d(d*RFKTnnLmf(;;FtL`Yo?i3-HlxC65-Xf094i-9r=#mnT?(<+iI{gq-74G9 zTz!J#6%RB03ad(Us)BM~<;x#W$mv6PJ~PdhI|N3okP1p7g&=NuX#P#aP7-OEG8jep zbEh7bYJ0aVi$UlUl?NP%?76oG6I!8I-diyl>z$Z3@>sCuMych>vW2Oe1g9_O#Je3e ztbrrICp}Wjiiq#w3-<{kA*ushy5%}sk>O%+e$X9yxlfwuWW`LUT+v;k5tlp>Qhjf* z8aXIc5lJHK^ilh7ZNkCx6JYo&(^74<)~T)0E>p0BDOj(PAmUV1b@+Uvg4tT`bj%Ev zqS>(PI8%`=y=Q}GW5`M9k@u6W8xT+znx}_3APEC0Jmu&1MI&5=PK*g*Jj8g@9bUCL z1;+w`5A@357U`*u+ZOKT08%{o)G`rb9IPL>rQuwiJ}1M);$#c2v|nkPwcE7YwL7#* zn)y^eextq`B zlVOyXni7?X`?=TLF{Xo&E+X?*lVBuLIP?}-$twB9sLI2LF~|d1Zwy{y<20L5k&Z4- zfbbT>aUIFrk}4J=P7aM_4Fd}pTxbr@jr^baS2Z#x8%uH+_=^uh|NqgG`r!|fHQ&~` zrw!-q{J&`0484aLClX5e*zz?PyP12%1PgYCO+o>4C)YVln)qoDcUrF`G9o`0$-I#U zBbEJA1Pw&2AZ5)2)-nT<)H+bfJ_R{OH6FtI6$v)7*8e$tfAARnfAPUonJnZvwv`Ob zV+Pxn{(oK;#B&l&QB#9Z(^cXEW=V&%N^(iU23|;0J%3AgTX#oyT_Z!!@H@ZX zkt;o;i}i7m`Av%R8DT;Ra(H-mV1Co&-2g{T|MlM#!hsZs^y1RW_<4N08N=m}$g~6q z+o&y&r>PAOw=k|85mVt(g`Cqp!#j1OdUISxj1(lPEjUFQux~NuAGmj4#uOE-oDDQ@ zH11zhgkL@bBM;wMH3hSAvhwz}pEhJ7MtX)!?hHhiytp5N^0*UVc(K6`y!}<(l;$EQ zVhW!GBXM{P?UEy%6G{rjaOskFASE|gH5Jp1%Ew7Ot8K`<}}Qv4W|u`qP-SfLEp*D@^y(%~}H zG8t9WhSIZEH(kRw_LS@Qi}VHaQ#RjD#dN2mZ7e2tuGf1pfc32Db(}&IsKcn~0EWl+xg^wt_=Q3F|Qv zgx8^1;gkv6rdj;*KB9FMzey_rmNM6G-DyB`UqvlmMiw4;kSvXPMEyQ3Y}cME&FJU; zllrQUFYiV-YNc07+fQQBLstoJsBrlHhATRh&Za!ceeK#aMWyITCb$pSoKU5_3ZZX8 z0ca))3l0sHZuD6&{@}9cXJ2g#$8$cEy?esiyX?3bEg?!BXT|8vFEjKgK#ZrA2T8(u zLf_(c^u?Rq!~F_6oY8hy9n)b$5+S0&N;4hu)Z%-P=HurI-XkGcLS{g`jTs-9YX0>C zX73{>j|#V6g9*0Kj8Hl#n|cqz2#tqfm!t75EC0a^ffZ0{lGeb$|LWwbGqnicdxD1V z<%<(uui6%gKC_}$cy5^eOA#ZCCM?aCvD>p>#ryvJTess-6iRm-nuWK7Dj`OCROmjF0cFS(dsob{x@IBWgcA-z&t zaEIp-3Vo;i62gK6GsF3_o23}OgK?XxAo>!?tK8w8B*iKNH7K+mMuDlx>R*0(Ol#1! zRX)>Hcn3iyJp4b@A^^YpXU-)^PD0~84|BjvxK7grBJsg*L+X6a4=)k1*2&8w`6~?n zw-)^Yjfm)L^p?>O>LnqA)*>G2}XKq0n7EOgk&k#cek+^ z&L$Sj<#Kuj5|=U^;*}iTl;D^IN=$}PFIfMQ5xhOdR^`HS{*z!NW<98vkW6W#z-Oa7@)Lrha!TgCf0TZ>QmWrk<-b%IP8HD>G1q zq-6A&l3E^_*(8YYBF6&Anku8(RL{OI7PsaKyiT2fTYdVE!M48@9<{p zHfwOy)zSP_liN9THZfu;LXflWy{-~35SjzCAPI5e`v=FhpFY7jcL;xlp$3vwZ7q56 z%C%>i+G1CVWZ))3jNB0>T?MB(yIsIEQKv)PAQ7=&t6u+P!p4m)82j+#AOv5@FnA&i zU!wH`;m#Hrp#b4IShtHoQ*8IOXfDx|bvrRc>@nf0G-tUYr6Ya{jH2Y{{oXwf&!H#N5?#IB?^U@Am|(Nx41 zvn6aPTgJ{~E1(iq8dn*W822QI1f!{uQ@E+RU##d%mz@}jQ-vf34u*dw3wH^)6REJQq*`Gvww>F; zU}R*JK@Zs_i;rLoXg{MB>kO+iUs|LpNFkM_Xn5z6d$UmYHMR9|i8kw=s~`Q?H)%DO zpB5H%Ek(7;Zff~3{@~65>DRc655+OD@{o@qci zpjos>nsV>Po%f~3UPpUuzJz(X=Faz|`+S4;V^4HTpKacXp6F|8ZDN_RHYZxNJf-Y+ zT6=2yIwRIcoeU8jI$8lc|Hz1vg`!0|*McN6Etp)=UW4fc2O-znue*9YQ-d6)sW7_A z6e!(bsPEME*+gWbDf4!m-Tb5*(b|chyFUP_T3^i!R1|Yq_e>DA-u!Vz4J}&ql8XA? zs-Zi2JIv^nWU#;XkX0{gLa}c9SjA_0s$G=TgZUyvefLG^_1z02u}r$+0gfgX&6_t0 z#g${hdOUo)b%&Lr@@sfdmnh~*HHkD&OE{&*8w|whEl9i&6mH3Ie`0W!a24#&p8&$=}>?S zWfQLNJ3kGB4^T(duo?MRCrJ+-MXQyUn)iQoi${1p#Zqqk)YgESb9H2`e7|9{xPeD$ zQ5kCF=m`6wYdqRVGfJ-e}l+V8;I z^<{l74Ao!--B)2=2b;hUPl4F$fnr*|_I($sCVQ*0fj+6?n&}X~)A}UzY3Q?1zb9sa zkS5v$M6mNAOpzzVR23rSaaWs=$2d#HILT3;9K@TFy>#g?_CjQap+_|WCVq5716$X) zumWQjjRo_*5V=&jf?`{Z_HyqRN5d1mp@OaM^5lBX@kDxHWjxOB1h#Uh z2U2O0e=2w#gRq=!Smx9r1O7FbFo=e_$X;T*Sr;qi64Wq1a!~pVNI=LKC*NvTyX*&@ z*((^#ubu_T%F>PsR*0;V4>Ga)RxUvv%kGw@!R6bf2PA`EdyuA2=+5qUYY z{^48{wJPkouMnDotIiL<);r}YB%TNFq`FF>;dr^ZU36P4Kb^O`vV{I zBOyhuhv|^;*ceQ;#e;t8yO90CPjgtL~AS_IW@jeQ>AUg)`AJ#w5l2lPX zwxmKd-nApEH<3&sQ-a`o<KAFr&eS30?O&G+HYb5BYWZ+AxiFGvTeb7TF0jrF5y7&? zc{#~f>QEt4s@EAh&oa((RFd}u!lJb-+FQeaW_)fu$rELH1wnI~PqQwJbe>ROubk~` zykFHeMXTp%2r1T^xB=F z7`7=8`UA4IAcx)Jj>lT8xQ3iv;6$5+H>hOy?%2?pzL zPJ@vsW~95qgqu1c0NgBByWpXSC4=Tb>K?NtxRVxF%; zfllznQv~D#iS8pS{I-(hX3M8pcLW_e5aS`t?J4yc9vGH|``%@~5GED~Ae}U$z_iyY zL&1uy6Oo9y&28k|U+mN=X@vAU!ksyD5n5pkO3b4AIa@njaF zvrVQg+*VF;EEpVuUbIU@^68e50R5WRnl9eMH*DBiWpFz7V^T~LU^tmIyKTgdV3O-~ zoUeX*D%`78@j!#a;`Xxn+vcKG8V_L-I7V_IUTF^%wPJOnBp6bvQEC?P8`xFHFJnRZ z7>L@-tz-pOsog`K`7r=F;mG|YR(}#ic*vrBMK8&nU11TWR&kM&i%*pZOqO87pj62a zaa40m^MZdV$3gR%u`R2SvujZQtOOtP+mK4>?GGBHI1YxjS=a08^lLr;5xxlNxFtix zjxgF(+GE8E7siwLHl$vEgQr^msSBlgoUCO$#Ng}eO32laYa#U^2c-Z-o{RM_OG3D) z<6!8v==vRQ0eU@AW2t$R<6&vVOfjzC6bDCicL7MgZwErwhnslw($ zeRru~#X~e%pEoIFmf~4+A1?&bq0V5Alp!Flld(*O(QSG!63vf!mV-m$Aas*%pEaBR z9|%*LB}8-}ap^F*rls5r$x&q~5OKxq^}`NxxN2&KzzE1Q5*W)YNVy(#Yb_B{zEN8) zaV@Ry;}WyyI0DgqVMzi@aY&;lu*AYS!~I;QE6atoZm5E?e%yNJY#7GKOmfii7k^@F za$-5@F%TthhCw~`1tRkwNN`$93Qc606r|1#QJeUT^V&OFM zN%E~aRKZ#_MwOMCzihT`eriH1B^e@osgwOytjkA1QZ@pN?N+qINdEmfkVLVRR6BF6 zaVLB7^{z@Cdbwc2fTr?S#+l2HU2uRFIi%=wBC(}qU zc`qJRA0sbMQrp=zOkqX#7O3HuXHk3Qe2duI&3j|9}k<%!>MgoM} zIR!@Z)76_#p=7kiJxN}^ivXy;XPv+4s}hs&-ot}dg$6mbenjlf6M?a7!BHHrc1SGh7#U z7B|?0Q#1u)iF!I^ZXRvnoLSOBb4RKX>-Fc16fF(exuKBoJHf53M)}TZmT3^@j_d;w zD=zx&^3}^j(94a7Ve=vmX%B0CIdh^hLHI^Rh6@4nr$OAU(A%Mx82_oJt~$dhlxnh) zkh4uXAuku$l0IW3bRwp+7}J7zd&5OpY`l^+9yi#`fWhh&K(GLEJorhakaaP5Ceh^ggN zpjXU|bw}n%W6G&wXbYK&tixRhGPacp2Rq3NWm!!K9~4jfIWBMiO1*?D+sXz*IG=5G zxDfIM+yLR z^|q;pvlstHdi7V-F=K8v3E!6PbBn5`u#wqve!)>hnn0JPi#t(xK3y6^B^%*?yYcn0 z95e&9)Sv5RDo$5cqnBGs{kf=nwY{m*m>1MBqbe^y`Ur{hQxS|VHZ$9XX{Y`T?U9x< z1BLq*An}HC)VKRd=97DN^maSwoGdRYz8oiwDW*P?Lwo3z*pGx_{)={_kIr2>FYPt) zBK61KaUDB2SNgI2)afo~n>JSFqu$dk)KSYdjSs)tq>)BtQqhpxnfoVBy(>NUc-o!r zFwG~$hiIDU5_{=aE+0yiZuD*1jVhSsmzL~9H(E)Zg70=eUiL52G*PK*_*=|ZPnY~f z`n7jy_xZ;5;s^gB-N!}UyB%!PLJ`BK&QPMER_M>{LT>pq5*es4BjaG`4vi|cS*!4# zP$P`So2Gr}`#^R43Sq8b_w+h@kMVIZv`n3SFkhI^Y;$Sx688~pyB6s-$P4*oQ^jmZ zQEB_5oT;V4`z0A7axlIFGyQ=({I`1^p3*C^3iNqOP>W5JY#kY#PlLF27-UIPhfeB$*qzzHV&z7fMlbu~^v+Fa(7#eq z(wbX`y_y-`LH|V?)S!7-Qe1)t{To%%gW2q;UXK?2pJ;n^9%I|)YD7oKV!D*5W^41Z z3&<9Khjy!G_L1x2W~8jyM7`BHX6fb&hZq@pOfl#wTT|USAI;R?(2T<}rcvDeXX%-K zO1n{^X3eX+xd`4#Mek>`^ABBLiXQ2^v_UI$y(hN3LZ<)MG+XpZ%cIMmN_LMfEt*_6 z|KSgi021|_>DM%0U$7p%{~X$U_puL3E>)rVE~d`+2KMsfo3-fE{U>eEa@O8<`Yf7( zzopH0F0=kjRl0PeIkX$yV~?(?=|VU9Hf_)%T|uXC9wDx1bY1%2?5Hk7>d_z3Cg2{o zyQjBUI{L&MkEThcfgttb%f{}%crP58_5|9XM(vIIejekDrTkzWTT-`f3vz(6JS;Vv zqMurxS)N-mHB-T{B zc%Cq9wZmn=dpi4~O8o0dXA5B^HUVjQ_|YQTHe4tdv5Zp?T>%5}XD-~3csf$ubH3BO z%e>nxn)m8_&Bo6dZ|{>5)V{KeaiNss7r`DEM4fy#q#TJ_$W?HaT)@MCKl59c9SOfs z_vJCJx2SCe+64bhvlR)?a&~@ATHig?MR}gR-F|fdxxj_gQCFFsg7Z(2ozJ6=dd}WH z)4UhCoQtRyY~-OQyQfHZI!}GQQ*8dqhn2|e{FXN8GJ7)P4x%^|{z#=bT=BdsL3*N1 z)L)y&ZaHhehs@6JX@i=%C$=R>Ze92vv>DjJ9=_7L5zRmYbp~ecsNDHIX$I=4&s3^8 z?x=cm^dCcSH-k3lFtg!_!+{2ULhT$dGz*OF)P~=p z1{PTd&ElkiRcmHq3^o&B*b+a6=IC{B)rcVo*OLY#O8uLPJ)L2$GSMUnqEPkW zxq?B=l=3*~?!8X~#Hw9>vu=2o!wHfxn*<|vhxs>u1*V5vYDS|H@^~2bFiL%V+jVn$ zzSAIi)Z<~;5pawT9ytUB)OFd&t=qPK z0orswpbgs3&1=3^g!al8bTz%TXJK2{!B0^EeN^?HA>orn zhJ-ql@jW36%&TRE1()=}=Ss5CRa2b#l9S=$EC~=U*J$(9?D!R%_8@iQ?^}`~;znd| zWHa23>M>zI!teY%2Pp=0FoViybYV7$So^iFwJR)?qB5z&l?axDQ3E=8c4x++m8HksR=3URG8b4V=XqXwaPE-Fg+m_YV*a%DGMA?Ul$30m62 zcy?enJHL9MT2?+DjmuK0(AHcw^xC&tu|C)75VtD`9;rq1n4ZKGLF>v6W-fq`)$;^xf^X)%@DY5~d-xt(dYJzkES<#tMough#N z-K#}NYO32-7hihp$w&lvAetv@KiX}}wV*oBco??FJU>K@($_huBuCmqPL6(Xm5Y(C zJQql=kg^H=3Dg~Z0@%5ucsED7ZKFh$Bk1K#LREjaFQZ0}@CL*?IXFBM3;gcmi z+Q~3#1$UG?WAbCpNVs5-){Kbdy^8paToWfLOCftx%MP?+`7(xzj8n3Ig_Y<@U=9IC zPXtSWPT4cH?o$DEor;Va>kqs2_85%cYc?dSvaphTJ{yGXOmCJm8tXi#QHUA#4%8y= zGu*xXTeZk!lVaZ!A;!l;Nd*@0^P}^gNSwHW#l2i<{E8$f|2y`>5*?nGGH&N8|b83+-bJyelDJ3(GQv<@Q+GV9VSjFK-R1pbb-S%%u$L2}u;c913Ke^L{ z(xVmBpL?Zw?RtVGeZEF5w(Eo)XC9$JZ_|EnGrRs;#v$~3xzwj>W5t>+FVU!S>QlX7 z3;HhE(DENgZ57r$-Mi@sO3QzXs-&+Qs6B)YN~1m0I;L_%M-M8IuA)9v9k=Vit9GWT0_0j3f6gQ9^wZSe@AzxDW-DcAsD4Q+Xxak+t zn2FRfkT(C`IjqR_0ae$Z-L^mfH`0w#<;j80*cY7}$6}EV;X;*1?%s0{P!5AAGS{aJ zJg!gi=f5o;*18e-%|`MD%OmT1o8qw|#&{TZB~m>bRUw$0%bh04Q-24>;-b(2(`zI& z_H9U|mATcEd*o?8`mr(KRU`;M4Mx=a&pGA@VM2zK@`-iF)X!P}Khn`@61GX1E0J+e zO)kle>hA_sO>7_P=KI)vvIIdZL!K;@{hro!`sDYeO?p-(r}j#>bn|j7nKZ;h>mVPb z`^A9x3v9JfZ&Vn$q<&93gAtTckx zay?Zjxvqz|EO0V7IGPeNUF$OCEVwcmfum*dPF|R6+j1?6EM;4HQDpg+mSI>A39*&? zri*z6576OnA30a{9*R_H?yRUS+O=PUiUI^mJS)o4<-#Z{RGYJjARrlr+%b4t+U^@L znb#~AD`vT{NPPw*9kssX&M~jJYR1WUdoZb8mjsgdZ5Wed45TmZQzJ2y?q{AOiop^_ ziPUXn4%OK(zkVoa1;*2>c4`CDdIGUqYu$011abG@Dm-v~j#w`wNv7Z}NN`)(TiO6A zBDmAJfvGKU#bPMd_(p|#1x2;q#%4LwW!VRR9PshgjW5u))Vm`@IdGVAkzrHkus)ll zlx20VR6oDLXEG<6&ZA_j)*a?}#VKUtbi>N`zK7+~QXt}(wZp3X`G!u`g+zr*-1d2# zj0cx@<9uUx;#Lh-hU}9GIlSrp=#A4$L7c&U^!tdCsdm=^b@|itGK`?rYo2 zQ4n>&V+9JL3ZpVLSsJC79j)X>^leexmPc&o{38d@SNS7t&;hn^Np2w;^!K!vDr25) z^lH?5pLaF&_x5o|=j}d-cEZnTw-Q+UjlLE%KTdkArrCF?r4U8-|3F6u&aQ^v+=uXt>%wuzQv>Zc$ zeGN7Izr^fW@pv;D)kyupoBXQtH_FiuPN34ovNw&&pckRu@y}^9p86c+mv`v6Ul1nufEnNQb zUB?g(a5X(U#greui;%7Y{mYc=wv}D%Ll5;aZO{(Q&D`zS7X21Ibb4y~>Op%A3Y|>! zVNUJIJFgd^VI!yrcFl&~NE&_?r6GjVl>rdQU zjovVgwiGIutyeaGEIrcKX*Vk4&i1aqj8@)9v_Tu$d-KoTK!bik`?WRfiFtQ_hIMgM zUapB*z4q{ZwEOB*g4%_;nw_V1qaPeayV(h5Meng|=w|uUnQGHF?Oc+HK1>{SlXA|= z+$>~X{*g9lon}k^I^+V0e@(^CYVL1cb_y+)$+SU>^-YU1Fj=Q5v_U1>)6KWBJB^_Y z`ieWY=kz`_QIqLSVb^kVcA$w`N?mvl*n?TmGmufprcP%y*V4ZAH2T5!XoI?pO{;Ho zvDl6wg7dvG>-T0zfRo{tkpS5d==wwTy5j9GevQ1`S&%qLXBeP6!&q{yO?!1$&Rq@8 zWT@26rDcScU3go(B%reR1Q_PaLQxY)wyCyTw(p?~)Er2u=7Ov;7}nG>TT0p@&@0kV zllq?RXD}Q#b*`7{`p@3MWWwL3?M+crw`}FFr7=sB#ZyjJ?K>xRf2~FS!r*$Rd;?tK{c9&bUNqqovAzT@BA6&a*vWPs{CIa z?B3pDl71_Up5AIt$!sW+!06NIgT!|(^*pkos<;xPwvfAV3$gQfNVH0l^E8O@a%8Lh;t|1ops`=ZI~&i2hY_O>mM)=nLft-?<3|h6TC{^Y#4)M6#ZirNvZh}V4QEla(|b2v1BeP zE{=ntOL$ePgt{`WH&bG4^`QJtS>v6cJGgwN=jk%Wdwn@$c`NAab?mm%RO#_7 zZ?cKGdryDKY<)FJ3Z>q}dR(q{G3TaSKbS!Pqo>9BY*1bf7OteAw`3qY%UP60}n&DbxSh4dwM)g0M znnVeI@@VZ+wDqH@5vX?HrL!GP`e(F3r_H+^4_ndCMbQRr*5Ax+cq|nZgx^pTB-*w| z^=pu6`$xJ3Ijy<3@$pj*8%Apj*O&A+H>1xtl-hWxJ#y>fYBXph{ReO9?3)%~OM#l# zXfm!`fBTr^^9U+-`LedtaTa^7SZc>$$X8j_8_*k$qTj12WbdiMMMf-@LXFm*b2ln3 z|0qyIdPCEgv3)>m(jLD2Kt&C%DoTyg8-HDRV%<14lR~$Ro)A( zFJWFD@CL_&Fb#UnLv!xwp@(>ekOB%>@b^oed$d9Z6AZNMkm;%Y%h%`&rqX||o@>0b zJ`XJ|Y8vsCVcRRQjl(LN)O%ghIv$D`ly96G_@^l5plH{lnWvuUrRTWI+wL#rP?BU4 zjJ(I*XZzR(tXd~ViLQ3F4eK_cKBq%_d}qN)^i{u3-%h*VF8iwU`&ceR0q4tl`&sGL z@u(nAq%qIGz3O4@1iW5Prh#5+mt1_T$9_VFVY(T2r!wa=EFvZHpe+TO@`9_e2yC4O zBe3iNHHj&F(2?T|m*S5x5VcxIAG%SPBvy!4sb6IbL^+K$M%vKlbrV+?y23F6)2L%A zckS$a>v^#=QW`on82yU%%$~+QcnCQI;xB505X4*KiV)YhB4hw7SCqUz%Ud6)ctCTz{I)sE{iER`-*4TE|bAXopO2M;vM)OQB%N z$#UuNw%e!L09w{WmuftOoe6uctHs&*X^pkp%}RwP`IIbP`*Y_aAzpfzM6lR7rIf-4 z_lmF#zSc4u60{Tkg5>s{&S;Ft#S zt-;_11ty~;79Fe8AeN>QCV#W}P5e?87|MN0``=?DF`&)pM#FKiWNNaxXXh^NCaiR)g~{kp{Afr5K~Z92qX&<0`S zF0A+@$JX;rX{Aks5pM6JM*jd~Gso4x^()!J-E;9y*T}iQ`23d-^okzyl=h4zEEYhF zfv9#J-9#)A{6A!ku*dCRK=8ffoT*bg%q?tjopJPS$V1=@UVr;jov+r{r?OiqN5zu!A zqU!ehu`5oOG0ruTkx5f&wT!HM_yiru;j)9#pA9pvJM4hEt-BAH@bfr9e_1at=ZGi8}K(mNW(6pTH4l%h?Nyv8mtCD|^F|2ZDQmucEKIo?<|X~Eie zWg$|Cmz7Z(plDLD^oHCF*{^BW9MBv@T0(>jU8-CnNKUBS$qL!hF=ukGsD{vNv%@SG znjoaf3=WYK9LNP{)a^6hrCGBYwNHBl8E%+wYB13NWXsnwHl%oF^@&bgHqg{U>;ju( zccGFpg&yT{2u_dEnc5%~i`%f(p+E{goE|iPs}}ztgSclycO54bOZucDL(0I&HD(DJn6ZA?SokkZS-TOW*U1t%9mBVM~vh!A%PL(vW$ z?-AfBG&V(0_$`zS9Z~#iGhe|flzqo|-czpQBPpCDI-6th9-pu~8n38Zv8hZuPn*SB zctOx)N{fMaf9Io`V;YizXG13|*ro8)6RbSel5$gLz*d0@q$c;hnqU)b2C}^{O~{t0 zY-26Mf&X&I0(K!=!CLt2zZVvv_Hd15EDRO33pEv*N)3fvf#@un4)JG=K}2gsf?04# zv3a_C1yXgR4A}xBlp$(y&XNF@a*ZNt)amPUzl|k$C&36W_C$3xD0+?txql?pB8&>4 zWQ2XIhz09o!RlU*bSc**Q-t|QM_6ka3C7cyl8PZiUVNfIZTJSGnu3`6_SOG* z#kv*_i>Z>)F)Ql@QgE;OQ7IyYfhIzXPtfU{btA770V7-Etf1TC({z$M7-Q5JbPSm# zjF5}k+gZ1R+7)p^FZau6kDI|~IML&V1vpIFS$kzVYF$f)h(G@6&&qN9)$chGHH(+$ z* z5fp>ZTPLT{k8oGhR9A&GRMh(Pl>1E4%8jPUteuNYTat=snf@lzd2B9P-=qeL!6E3n zGNp-en*plKx(mq6V0`zt_Fi&ok^c}&2+tYpm)Hk~ zojz&?FJ++Q_aO47N6gaB8mvD!5r&9bN^>MJ2XPSUOG8UwtCZ`z+fWXNcmH6mk!+w; zxMRr>nZcj8O3JP{%1l=w?Xoa;a6;c@{JB}S8W$!U9}i(NOXXP?iz2!ytD+Al5T^#m}4?j8(TBlj7!90BVaX?+=`9HH?aB1<^AF?`P zt754bvIR~+&C}kh)&=WD>#8WJm0uyU76)MiCNhvF#9hZgBp6Ng+6GdK-LSKY?Lgdo z9`N;W53S`D?n^D9Xzuykr21?;gxNyKt)B&poGmQMSI>kb|7ZUrqi-WS|5|h+tD#ku zdgqTM*s745tsmt6l$miP!TxUbvd^(rW2#hde@O4;t6F3mk|AP%p@BD|W zB!a+$-#2o>37Z!AvNBKaV6>X2=Y5AO8tOZ659*UJKo&hr$e zOzq}mDr>=gEdJUbok+sgW5}%DK*=jU-OS)+*WYztLDIb5cKk0xF6u7n6e%bEN_^>+ zGg9sSTYg;Xc?Q9O32?_j*d@5nu4UFS8$#+sj+)$lR{rc}$&z<4e!S3Jfe}}W8ZcU(%Y4exy^Z30(M1yzplPfm?EQLCX>W#egHxrV(%u=HnL6Lct45(!k12TRC znSit6<0q>QB@$Z6WT@!dJNu1C`;eU&>h@n%+-IHcO80j#!{%>G>iL`d5#UUSl#8^= zhU%Dy9#79Z@~0%Xhy1ogb|6Lh`o7=~$}UJG6zZkY@ty*nk#bkBU>OxcBo=2dJ(fs~)@@ce}5lvfF^W@VM1<}kljq4AuZ zmeGKm58mw@SC|QZp}I-aYc?MK?C`A5)&Ty>mBkW0W zlHvY;fzOklE-bxhHjt9bX@XbAC2%1Z}ph4y~-L!GQa z&V1CoAse34_w-CfPUW86*V7IT}q!`x-=G54i*9J0eE z04q4rrjrWMM}WRz40$3rvz%J&OIS8VDhYCk_--s%@zT%H5YZ^jfuW{JCuB*4cOTQE zg$Jp&MY(2l$h;3}1cOrGWGZCXax~5o;a%*xJc8oQ{35M_`uHNVB4Z_stzDu@Cp!d{ zR8VRG?fOr<5CeT5V;Kj-+zap^=kv{OXz(RS(AuX#j55b0>jkP{FS>H#QF18}!V!_L z^m1F66HRFEdI{tH@P(uu%WoQ5b+t!>)|%(fEgIjS&*&UqIkHtaQ3f5QL@SRLV znMpN6xnE!1bA?0Dk}*m@#Wz7oms-{$-%ArLXIRw`+forL=_G-rncK!~ghs&QqYQx& zP!J$6>K@CPR3RG;+Ou8qDSL52qispuC^Ke2U2s+sZbveq#rHSzMn)Le=LX z9KRPPAHpyFvojy%X<17=gx7kG+ zpO!A`J&y^})%U@u^S2=-+KJ3&z*AyFcKXWQI6cgn8(J3hzuIUMZn z%-s9E-|{I81~Ds!j6|3AS@t@Ig1jKS&r70RZq+b8TCD8lSAS8XbSmCQgIIF@+%}mC zHOUxgpa|ScX%aTV{qo2oL{5qc1?mpN_*!b;)tQPthm@VFGB~O5Zdxvsf3^N3rJlg( z8r<-K2~vZMEq+^S4T=^{#u=mW3L0(u6?OBji{jVh5bn+rrpf@GU4F$hw&d&i`_Iri^UDJjO7A+Bd2;g|KoK zJ7-j#WYP|z3^Jct_d9gj^L_1f4;rOTjGRcfJhwglo;#j%R3CMoI^(%W-Syn_h(bVe zZj-_Y_BH}Mk<;1lu5xQD`TCGWt`AwzPtbj2q*!Sl*t^98qEe&r9Yt*$ZUfrWd0;hO8sU*FGkrd2uM?hhQynvvbrYt=3dnkH)n(Q2-E@1P8+T7B2M<0tR5usYc#zRB zbgCr*ut~t4@TEyU2h+%0y72a|u>=7d@Tk+jj(?MbVa$(1WuB%kVY>JMCA|tWChwNC z&nY#SmGhr}r=v-UPa#IkK^PqV>05HOa<`$#9i&HHYlzfJXb4i$5~W`q&FKt&S0wps zn24G$LxA7w=;I};-zo}C(ufYfY$a-=jTWPKci18ae5#(iQWPiJc-%oouJ)m+dmh-Q z<5Om?a?__P@EvPXzI=Dtne((d#p+`4v@o$qW>_}wM$i7uq&41TE`lXWe$b$ex!y;AzfYow+7+ZDt~Xk#+}$N6L0v)ne2O za7K#R{k-;(5VidG;SIGlY66#??5Mli?g$4IDeowYED>IwcviR)zpBggaQPArE$X5; zpapYLiaZeZ%_+-#DA6~ZtP@2~5r`O)LRmpX`kN2VJ(+tmA2=R59yuO61{_)~tG81H zfC=@-EgSr8ZnUAXnpL&&H8uMf$p6HX?75Qjo)So3>Clcag=)zqOA<<*ia~s6*R}~s zQ||(BKXBa@?g>vl4{S>PsjkM#W6ii;>tAYIBiWq&JwB{k6|bcqaDN`QYFsJFq3|1! zAD`aeWNtAAzLea546a&Cskq6)&wugPd$?G5BkkmJ$>2al+d^=It<9o_AhPt9=l?$? zfE1=G$X>(D=&hnraaX7nLgyDzz55nX*h%IC=TT5s`+labV@u=$_bI>(zccn?Nnl2EPIsu2MfaQs_Ezf*EAsqZC;i<(lS0n( z8N*b6@o7_U8m7=O0=)M!8eq4uhS4rpO+KDt`-*(iWsrsR&wJ7^^>Y%ilLz)#z^~)# zxmr%-8`#4Q`)L($qu-g4sey58qQxl4S?@9KFE`JlKiM4kK6<1@;GV8+F_Kzi-+*#; zoHLDep^cP0RVln=I#OkS>Z#>1z<8~cb4Lmx#7`yU4rD&2&$>HY>s=dMmt9v}gbx9E zM(Va|_?h&3ftnGHmB^!pshq(1X$5kflFWvgNVNalgD)jwV3rH^N6Z58)6ABv zFFf;|B7b5wHK_cG5Ab&6#`IM@YQ?EbiXxrCm$3CV)q{|CAu3XMU|_Ti4ZDV~=36B{ z*4{6LjL2)gX5B`s!*?fhB#d6bgh;;f!b5s@s2vh~;M{Ff_n8U7zye6ad$iG~&4#Pn zQ93*_>g!-sW@%D~@ldvz(V6t=w!`|`g|n!|;1Tm7En#)}GeiVX%_dDSJ?*fobFl>l z(RZM90iPPqM1wgq_9mG{P1!O59e^hIXA(a#!VEah_ZH3l($AureI$5e?b|M1>m;g0 zc*Yo9AN)i&#VVb@SqUV+GoJXY`)RmEX^)2)aU2w>1T76u!ef@XFJ&w?3ZB~!4Q=gLgUSRthz4% z=R=!%g@@*nqqL@r@vxeGAF4XO1Vs@tE6$bPawk9IQ$Dj7&q}C8fl-`As{uKZE`jsxr<>S@6)qkC;|NN|bdo8;HUF)#F9;??BCH}0ssw;ET z(VDsA(VxW$hFj<+mUJYp?CH44U@CB$hRoOnQwH!M@ets@^aLE$jv9w3tnQ`5h+|!; zGysT9fM@`i*d3AImal_YAgF$MU{|6t4+u`J6W>y>8;-_EV#2v0tIk!`j+LF7tg*vqi>Q<%Qr{Vqa>HwImP)h3!Mm z`jwFrY=%XTd{gc7ha@cXii5m<^5=h{dCL4z%gd#!6)D43Dp3>R7o&*u>sNJ{y$BtH z#Ky`j5}`adU9Ovm=GDa@5|Fy+Pi?Ty_(WE>-eZXgd3>(3c0Fywx+LSE;2@VA@OJbX zb?lSPRg|hHDS6du6zAHRK{O_>B(YvwN~9_1H%6)3ej7H(`qt+5WjMIO1WmO4BGpqR zBSnbe;JG2(6mAKTN{*N#YwUX8hLRO&q!PYsGbLTt!(+%163A*OX&zI3yQjN>3N69r3mQg15Bhl*?_O#O1&xHk)+vS7vq;soE*^MmtMw&)W&C4GNL#n*SJPs0M zJS=vg#HTv^)!+GdDZ(036iBYRXJy+i`-|GDeOM$NWey~7kS$V!jLniqLJt0p$)jr0 z-&fy6ftnXzgWRQzqT!=E?_Jl+4xkxvWxO+4&+4!cV^r2s83D^eOyc$GjonVW=Iq|5 zBn?CKYY+Y^KXA6+ig$@!(=7jkYz?rR`%oB(6ev1^KP>&#S>)>-)jF$YY8YLiB|=8$ z@uR;S^tb&`+2C(Mv61W*@t5;f{oS28VI&TiifGTGgO=c0g$5gbH6_&~4K8~E zjJTA!&S@b2pz{Gczof1np8sE{d-4txCyYXpC^mIWlJcdMRkK(nw!LmDS!Q2r`Koo2 z;|2Zl)>A(ITB6zkMiNoPbGj}4dA%GOg9{=h)vv7Wp`_*zhWi;0Wd>iT6H{Lt|3BL< zhkvMiP(?GTLJ-O`P;v_N7}ld}liG5GU3 zx`R$+Nt#x29yoScRBBN_xF-Dy8#xt$TO9+URZjZiU>zz1N+UrVBNs-4cCF+JqOnM{ zby4kImmcd5tncNwRfUj_mM#t_~sSMAai@hD?VhE1R}_h z#;`G5iNl%Za*Nl1?5PEqz4AV+37WLMN)jLqDY27Y4<2e`QD&~086Ftbp^Vr%qGDkQ z5Ix>seO@33WlY&6cCwJYF4O@JxT==OSWDq;D5>%8u1t3LS3Ndt-}huD^!?nDW8tC%P(CmPQ%%-J%}S3b02h3&|C3M#TK}-kv9~ zp%*?Ie0O+pIgWdm>^%IdxDDkPuM83iR%}iwPH>g#o$(E%xJ%@q+iWE zZNaLp7$ID};1mePEo#Q8i6}A^g1FYX*CJaE+78*UL70jsRtgz0-Ws`s>xMQApu2TN zBFS~z4ckrIa?1+KN{d?afS+$~ITZxd71z!9?K^IT?YOOEKBR49x3fFgo$M|)e$gpg z2Lcn&sp15x=rf1VdGmoWj%)I;aB~QYDCLl~iQmklTSC)e60%vWczhH1Y>imb#1D^J zp0k8-G*`z|wN5}I3H@~eUrj!T8h<0i0!T|dNaZ~{ubt|`?KdzzDBds8A?3&T#5bYf za@LisWDx9H{~XJmVCdU;)tEpQg)@ojl<#uv>Zr1y$~{)~#sFwFJU*)CrAq&Xfpj!Q zGN0PiGB*O(v8*9kxIf{Q8YQee&)3f1Sj*#&|3Z15qWCtHZsMPEq*|+P&x#0zh92*J z%d0!RN}AFeMGR|+x6u=~?}u_@7&|}=*XiQ4rn^XCE-HQ&lhq*iDXG6v4C(8wiI|8* zIs(+}BNH<2zpZKi0;dDV1%_r@=a3>HY<_Rmx3Hj8Y9s}7Q&_RQnMYA!L|gz8E6%YY z9H}Hx5sTYP`1<`5^+`+9>Do4 zB?aXl6Kz_@>n7Hkp;#8*Op z7up?zh>;K#WcbYa%&AUFRAcLRpoHiE8dhTgnUVrBMt3XHlbY8|myO2={2<#n8-cdQ zh(iI6(F$6QJMQgK_F0HB86ozGRqbIcS1p5#KD$l_2j)mURF7YB;Cd8B3+`gOWuRIq zcqYdCzc(y5Et+L4>{&*K=P4>g(FPVl$5khA`>CVtevD|94VDIP6xS@-I|8x=SCmQx z@kO55h2V-cvOaa8X?Klm4OfL}oU=aHHqSQSw!pTBA@N)!Sz#DJqZd6Kva4zmG(P8_ zwBBHFWS-fhVSn4!GXdR5h1qO_Ky&IrM%-8LeZgY3KuJ1w z@exi-0B7l4)x(wWnh{y!Sg(@X$jXR}WHrcSylYMl3fNtEfbHa)yf0iDG+HSx`=Bb1 ztj#?ktUMYA#V`0&P;{g~e=(#_{P0%~t>gfd(6lar^jf=GZYgH@A=4qGQ!zoTgxqMI zyqI$vKx#=16C<`I$u-X-#K-vV5v|uyAuWpEgyEMmmuK&G5!ycn0vP-Owy|w7#yhTn zta+kHA~{JfoRV2A3MMNVwxT0 z2|i+WtX29cGn@#JOgT+ZQ@6g((KEv+g{+L9kc4(K$~e#cE*uD9=Y+}QTu8l|HHE?E z1H%CNu*gPEQ%AwjCJT;UB*$vKAft!Lh$SE%u=u$uPCaL82+>jt>i5%P5r~m`Amx!#+$LZ&NK6Cr`>Nwl>~_%mG5oU?A2x&V1|E}-st&N&Z!o?1U)vk{DOCcVwaB7N^CG6-C6F=6`-s&F;1J5M`UAT)^Dd)@dYKX@ zz$ik7Cn>f~gG1I`o5VpC#cL63GkTrS-;J@7qQUW8X2A1D2gi4zd|%4))EaHK3d3;8 zcT%tXoj66I0Mceyh%PUw98hL30Tj;mQG@jg6RwBSZ4$7}dyk-R^ywHDT41}#5>hY0j zeIbawF3q|o%9e`QsiBt}+4i6MatMBx<&f3k)Ia?GZ11sy_~6&%#m}dvZu9e)K3KW& znfhxJK zXW<;mupnotx~fX$(F_&ybp{OV@R=PxbViJToaxR5)`ixPwavPSbd(^)o`#z|L(X#N z(OeXR3d|ok{d{mZ)~Xm`_QBMS`g#iQB%6LRb>_j}D4+aO^PkL|?8T1O?#)Ym*(N-7jtLj-RR;W<`>&HFFb)Rc8kTj^%&JX{VsZDcPX=IXm73lI8(W{ zpVB=T=*E?&8Tjx*QrV09SY-DKb6~v4iVN~F7Y5t{0tr!W!@B`ndUe+U~a-m~qP-3u+ zZFpg}zYfFyml^&<2wgdfdk7bSIA800!VcYh$cJ%}j3X4e+DZ+AGG~>xKBCmO94(c# z7oKi;jxMYeeULVdjPPpYXn_uKlcgI%Be03P9vz-#YF-v8uT_YI#K{ihkVv5CM?lUC zyYbcOt%ts?!rHrTS%@|sNmkQz1*+(F$&U^04R-vw)_w}##pOnG3C{5>AeAbYN{rd1 zPiG%US1y(HV}s+t?A_VwuQrMe{B~;b>_y7Wd=%N2Mqz|d>o;8VByPlL(PzGxzm~(M zZH6D1e`kSg!!9h*hZ%B?<#Ptn7f&#}&bD>!bsEdgjkoX8!5$9VmU^lF-);*`NyoCO zvS?!uN1>wqJE1Y;L+i$P`x-4Bjbc2i)v~dr5=us9dO2ZU<<>P{_*BczFAfio~-o77v4-12a!Oe2A7g^m$ z*NstXdUNO7*lRWng7o_8>6B`L>0ih)2NwPhY=Mo0oSotV>n<^5-7PA;bLFPH3wM99 z61(Xn5zo=MC|`wq&1rBMUd=zqop$<=4So19fVFAs(&D-h{QkVz_wLy6?>0**n>HM> zD4A~zvSD)d;)fT|k^iqrE?zFxG8mP_JoS4q?K27^lbBvihc`B%i{DA|$u}Fepo>Q( z`ORBjw4vot9H~3*Wum$xean3xHkTd(P=o#JpB|j!L&20NS%_CW6kxLPziJMsr)3X! ztV%=gv_xNmL<1=MdR%#p7o6TS#knoK0(Bh58Mfk=3oB0{Ro0S7Drq!jyL^uhPg2^U zGL@=SVU1XeK8lHHUm>t=wPI|VHP#4 zlZ+urWyUe!yKd3Cy{YT-O=If@5xU+k=7N2Tv)BExdnqh~<)C3K=q{wG&yg3bltVJP z)fjiukt!GL$1)bA2fb6hE#4SeUj@`D1hGLY_RqdwH`j-zkeJ@jG%scWmT}=wfJG~J zEEH_Ft5J`7syE%%&%)O+x8wf?BQMgI=pX5866t&b3F7+FX$aVeh^!3!cl0 z3C%Km7P!_j`6&|h_@Ouka&KmZ_^X^Ds>#+|wT16%3)dnDmxf09%jV5XC=8>iKg!JW zKRaZ_1W*cN7CWjbf<_FmQ17EyZt&&*#rOptV5fU8GBM;9)1kEBBP&pA_QYD2w-#X# zonjDG+-TP_IOo>VKMO%@c3n*0%Bwvd)CxqeK`t^^nPb;;j;oa{a<#-@?I|O+6L;4t zH7T*drsUt`eDHU@!A4qjjyRBcP!uLkh{F|t6Js4mq2?#3`4{if>UXGP2v*IUTfZ2? zNfqe=F3DP?B8F(~0lk47q03qUo^wKCj7@H2vFeFWg$361|4H8ym7owwVb1|}tvz8M zN(efC`;Xa%q!L3JWVOI!x|RCg_a)a0r^<#{9l6Sl#2vKuc6s1zVAcBB>fU38A*qn> zr`B~=(^$@(+@#?ib?)p^8tc27M(mw`BTq%-|41(3MnF0x|5kfPnuB7u|7qjeHVz#+ zBO#~Ndx|&m?;xcN#zMgi?z`W+XMHE36`+f7B2xx~l9R9TRc(+l5y{secZSy-nYC;q z1nXs1nV*;_Zl8yyu!L#a4+jrY{Y%E7sD!*2OJM~V0xEu0-ReVmVtGv9T?@_=R&AvB zbt=T{Fhh#bt7Bj%wXe~KV{+sAeJa;$Qi{6}Tq@bd==0dE6fX6=z<*K}x#n1Hev%DISq%kSg z59~iKI5q+0(QiRff^=s(=7xdZ9aOMcD#Rr+W2fjVMzZ)VSTx?dPibq&QC1G_hM8ax z{Ak^x=`1!I8x6ybSdUtdS&h7t$+7?Fq?QuIb{`2jsy9afVo6|Mq2AffT6eUfDPfvE z^6pya9%Wi9eN&S!YNX}#^F{Jw%E5yx7^w8U*nc~>@IwTBGBj)TPY5a5flP_7LtdwI zx+BI8)aUB)_ud}}hA;)9VPNY|!|lL|!bM<6N5;m$ECgtR$7 zAZ@g6IHrrOVuI^TI-8Y3#vJbyel1ICypR_b>eyTe9lcBtp`s=x#MvKky22RE~gsG(Y)r0q?1rp}{ORRxU8NYp=m{5Wvoe1;6 zIoCVSi}!6w0q1d09J9VodXf;$84s`vvqRZ27dzFzj7ep0aZ9LuXuj(e--6=#-cQp$ zOM8~~Jk6ZWLU3E(O4&%6WGxkFOp1k|OkKYaO7OK}7WafIwK4(vaB1%ZL(nPn{~_|LW0V=UVI4=154ZE zzf^)zx>!cKWhc6o)iM>XMQBqq#uJyeU{19K!x83zy^UWCOJFI)8jcMT8?IFdKobqM zYcy{nAy2-NH<2gTbckatZ33fjbo%#*?JV2-JoAH>nfze|n^7Yj6-Vj1% zS_mt@bKsvPVI;0RuB($QE=7DHfg!@cWNe+vEQ%?EiwAd%iOh zEum2`WU)BYHp^z<7afr3`al>dP3i3P88W@-R>+3<6yJl_w>njp(kYort~|HQdTCwg z56aqn2g)#LlUsR0ToDPCK5i(Gl<*mBWDF<9@9e>H?do3LJ3%C zkcxtS8JRu!!12ibSip4embFk}l!`CfeQ?t{ww`>BM5RHJG`Su0& zFKwZ;wzNfQi_@M9gTm*+7s3nSOF`te4-Hi%t`5;kD*b#Bb*EZMrK74HZplu1+X;0N z9h#&_2gKyNj9h3)hL>LYn%1yDc9@S;!K^CzT# z4C#Ml`Cbvr!&QWEC0YgrW!RI9~=mk2m#@h0S-<3Xto=MN5 zHTj{QL;7{Vue#PLwRKjoP{caxGDe5i#{4B{l6eKt2||9Uiuu^H+PXcPgX9Kg5s z(lV{>NU8mFbdvKsu0(Q9I(?4-&mymp_UfVelw2!e{1*LU-1hut3ynRu6s1BIIHeW| zE2@T@Nm6i!nC=zIO%91;+YrbB4ZAdBxFQe{hYBrIYT@KjN-*<^uO-bh&rx5spF|{n zF6z=o%s}hd&(Ggv5U?Z;6zH>l>^SE@)OFI9wHQQ9lk{hZMr{cqW?~fnJSR)$7JHX? zmwIt`jE138tcTJJPeXpXP&E%dHK#ZR z@`_7J%gQSv|5?aRwrUGJVaYyOp>0CGwP2DQgKJ;pY+i9Du4B2IC)K1qzw)RO;UR{Zg^hL)T z(lLhR5a4^oZbz&fs*+#@(t&lE>MXJLPlErfe2`#fvNxIDxuzfYQCB_dgmWJGS7&Vg z84^}AdV!tG$5!u|+17fb@9;t7CL%{o%uJr7sTYHY#vqv0E75iIfwP$<_N=9P$wc%# z%OUF|GdWvF*;=DE{W|zgyONmGl4s&uP`bjq(tC+dt|QGIF50FfS06=^n%macSY=gu zGnQfyz{fF=H}0*s%~ebnZDe{H6;NL&l9yy%Wp=q@Y*MwAoLWOmVH;eU(?L`9#Mb%q-rA*$3r*q^`obrS%_wAPP0qD)423gbg9kk(s!>1Um_+z)cpCd^*9~*rD|=3rL}60zCMdR1Nt_JY=~VRv0t#O6;@W&pV=A`a@&0n$O<~96}gWxHb|$@r1kMl_1ylbC?kmOKKt4F1x@HTw$%Iy1*Z_~E@tA%S3Rinmu=TE& z+`+1l1skmR0$j2zl{_$YVch$+bg>v2qa3n6=Jc6qH9wbTpipXwlo3JkjPOtYtRt&F z44}?lZ~wv`^9LcjiX{ez6SvALA$ML@^nrbfwMtx(m8hK0Hc>%@V(Os8KY=%{GOAW> zigN@_f3X-co?91Nl8J`sSKp3PmtI=2$EXXZ`EuGORFaqo5WG?MsFA@++#-E6eA?dQ z-%+lLPCYdcX+GeSXN9rD-b5^|XEq~ONvO`%G@z>`A&96JKCu#p#lZ+hneE;8I5@pa6fNovTL@Xq93qq+fx@^eSpKwMTetRSM@t>!}sUHlc@# zWO`q^=D=Lg9um61h4sRjp8ZM}p;8>33}H*k{TaUB5y$ULW3ZCR@WQN*4_>7(bfAg6 z4r#xT&0kC72W(oWjWWnu%f$Kw9kwTM*mf7zQPIss_9jM{;Dd!dr>I$IV%8C@<+`~f zrH)V~(Gr>+M3D(Ivd>#CSV$?BDk)<+szCV^;LXda3;U3$Oe+;@Co6*^J}@CP2?1Xi zaRdyBHHG%Dds&$!m5bSb3{3N(hJ}eeQ{JjikE3~@D87?24otY-NL8Ic%?CD8nI%IR z2}gSA8rknce>4(?=)kTKhKQ(&$_(;-Vb?`UZj`V|#V~-qjLhs~_A{M~$Twy09zeS5 zK11plIpi&Ybj(`>Z?0+lNO+OFxDX=Gr7so>3UyV-=uOV?8 zCS*5_F6$8B*JsJXG;8FSl2)l1{2H$)Je{4_F-uOzuo$k`(mP{0D8?sZ#CjVCMUUvm z^Z@-NS)R_coRu(~nh%_l$d;qJGaVC>Rn3c)m{;YK>Lm(4;#IJ1RiC01GFq)OtTU~| zPtrIf8mjuv9U~#P8krVz37K=Q!9A$FmZ;adR%%Ju&`S*j6-ZTxj^x}oTx-3gi+#QH zoi0hqpm-x|9DY94-?pqr0c4A!YsFG*B&+&)q%g|K?z#_E4k9#M%ADEiXmMSwlXJg|53I4I#% zuXkWxC>_s{rAxa^=@WLH_u z-#ODBq%r)>aI<|y-brdm*IV?48v_;$mpK0G=m3&Qa5-fEJZPJ0Yq4#zlDDmGF6g7P z+8U_9U715wjA6&Mg__y%C`)-h8Hq=!gW2PXZyo!T#**uAe60>1ADAfB%GmIK2nfIV zZ2}2Jk|Bqxa#8Mx#Xorhx^Xq|f!{tvYL(R!i5N&qapL24x<7j|VWfuY zvu>~-5o9(-IC*Xxw4LEqloLs^R4ZLZUL0krrYlH`>5fH;*<2K)qI>3R?4-T{6*?71 z$}6`R#Ka6bY`O($(%bL>Ki3@TOm=5=>xoCP-dx0L@-}ksW@+sj~)! zvummSr`M04 z?0Bs3PmmNij-)TKZfShA2sICiBM(VuR7!$>viS{OzF*Wfh6=#7(Mxa*c)w&>qHJ+b z0$8O>CFlDp;S?b#j5MoDCP^;2T4I%>%tV@fRi>n9j!TR< z41}L6Ks%u!NJ&djN8J)1y`Z~M0@8zHRH#W=jssMrRKye?(D~9J(V^+eM~3Lg{TMhY z;Iv_N7W*3HcAJw`H8fu{%DL9~P3MK2G?cR6_>giKhP^mtxN+c(wG2#ZeKJ^4|WB@ciTGXO#YCiqNtu4xI>jVE@f;&k6^W3$(#A{W5ihQmpwCzebJx z9%&{HA`Gmj+O{dVuWS)Vkfn&4VtGD2q{Pk;6WI>x)^RlRd3$dk5`C8$Eq{H@=o{VF zv0K%pR=uzuoxC42TrY#HTh3qnb!=L$4pRiNKQ>iX$-H5BVKo6QIr4Q8-FE$vBCjiP zrX3-&kvj^8;&9{^{+i>yYXxr@VND#Lbe!t-+q=~hq@aY&i%TFqfl!@d`;2?W#W}I$ z2Mz&#zx4tAkdB;#q>fZ+bhNHFrYGDt^*jak{%vSJ;ub|0Su6?FuP_OS{5w$ApE8?g zadw}QqjexO*j)oyR5};UJ(N0OTP&yekC!Ca~^T|S; z1b3>!$X0o{$+4nwdoP4E-0C+PU|Xq0dshlrs4s!^MB;Kh1DkP^&~zijA}K6Z2ASh@ zSqwsxYzxkfVYDk`@<5Qk{LiNqUz}Aj6N~U6uJlsI`L*bDyV0mJ} z;*uu+H1MH*#z-d?VS0EviehAl)9=e?Ts+)@Z;2Afm`qKfw8}nydg`HxsIWJ))g_vN zhI%suOxM5#nsn~RWDp-_pj_~?AzwaS!lS4<9I|76a>v-?gcya;EU~kuD58@=6Ruq= zHRaiPrNJtve;xgirGGzA-7liL`fCWLeDdSIu3Jv@r{&;LyF=lGDz&I{3<&1vu&=l4 z5M)X7&I-uBF6uue4TmgYUrAE!OCWuvLubT9=3YKA0i_|C>x{cY+#2VN-S_Nd2DT}Y+_*zcW(=GWOkQs)-n-$Ln|MtE{b}nmQ(+h#0RO3zIK*BeXyeaFZ!6qw`qoc$T8hsFTr(5mzU|=GoAxP?} zgKAop<&YIgX_m1XEU)I|eLb#Tn0Qxu;D)R_Up~Y|VuoTc6XE4HYTphSD+SG5@ESr< zEuR=Tdo6{)OÞVS*w)9nFFn8w^&Z_)B*^r=mhdUUJ0mZeSv{#98bzc>E=r`(DK z;dCqqt1$84W2rp@|6Td5zc52`)5i~PLtTNNncWuC-r9on4LeL90aJCW9oZI6qRaqr zOzX}I=*fO$>O{Eq(Gm1yk0fy-q_@@2qcEzw*#|$k9Y7!KkG?W=?E&>Gn|}5Gue^xg zk}ra|cJ7M5Q?C7m*{AYdcm9|1TOHysu+IelH!b#|cENLA`a@A|tpo zm&DLo6Rlz`ef2yN^~g!_^mm}N&pDf!!=%s=-SfSgM+k=p`r>ZH-!y&@=#en&o-S=M zZ{yyV!<5JTeH+Slwu*A{{JvF$lKsAl$Y^PEAY zxfqIe>AHEHg(!sEht*ffR;34I?j)*_V;c!MH=TwpMq#$}_rZW{R}#c>L%t0^e~dI8 zoi^4;64>w5j7Rx^t7?giF0C~u6Cr<|I>sSz79Zz{qU9iAr@sPz{u}>J$)at zZ&3zW$7tPl8@DPmtfVMW`xFCm$^)vQ0RifJDX|nXA6R1QNYlku5zEnu>}E6!+3H;& zd?FZ#%QVcQq-9}jO5+7~n|G#trw-@gt*FRjDhw}D3Z__pphId^c=&k&@3)G=rdB32 zIgE`&i$L6zsX04%zhnt2G0F8uDrL7(%^N2mfLJ~__h)RhZL(olNy4I`%mD7rU!g7^ z+zc4>(#WVNYwpQO=)|!b1L(*1e^HH?6s(c)2A8(&@m>o1z+qsUgoz|^e%3#0B6`xM zvrz8$wf$3AtWR5NP)u__4$eJp!{~;kkkOxcCvzLz&$?#E4+F9y8g62#&J`})@HvHj zeVLIO*)h7|>|Pd2NlXd#+>48!(3sNjaPUakM|g?V8=hQh7ka8>)DkPy^A;la?-y90$cay(vy6RgcM}&q0i}i$TO6XkrJ}=G5Oo6}?NVm_4I> z-$GOfb4edUfUiG zV+bbS$s7q^qfoETgvd-{-lC&z6$b!H?U8I9bF4%W@_hzGSOhHQ4 zb6~Jlz+&QP$W0JCc6;^RSJgC+QloP5#INOrGB&AtEt#me(0iT=wT3V`t#MG)NvG0W zC3kiiWF>^Q$kqv=grTdyOr-VA|4@AgQJi@3jg&yh=h(VkA#5R!G++26qu5YHYn`?0 zX&W}oEdsHN(Q3LAbybm$>7DSTo9dXUJn%3b2My{j^rcjL^AQ^BGmkVyQ!S!;_aQ)@ zA>3nqam`s78=~lqJjtr6LOq;PPt^%kb!;}JD0l{_)wR_b*Z`nFdlC1CeJ@_xuym?G zJLKa2e*N7iR*d92GKo7w7uC?Nyy1#?RMB(W(GXcjLJn%mX&$z)u9qF~%QU86l8zl) zD5_j&XToUMHqij8L|R0A3r0V)W9Sv!M9K(5)_vP~s3{~UH{b*NbMAkuoW&`5l8otX z6oLCoO=YX$$h-l*&mFiMK+WOLtW!E?~d0Yi} z_81Aak=Odbu?O}72EtjLDNRBLDys+ZCsq;BKJRxOdm6?%$wd2n%DdAB_Hih!&;tUI z?SVY7|Nf)1La&hcklDX!n{~sk`VcZFaa*!IH+%!R|n}Ci3 zBV2yt#QlduR&-swO)^t!98G?r-W%%sq1+h8W+Ww$zKPvRYXm$=*`i`d$FLyih-@hs zPzt93N-WUmT=9Np)1@vBD+tGcZ?88QJhxVI1yQl#5j|j?r(NLzALp!n4oh3r1jgUXYBK02ihd*^+SGT6zz&6Tq=Ny{;Bsa z3_r_VX09+{>is`Gz$ImQt0l&A!QSY)=BRVW$Jo4I;_H=!pEnlP<^H{DU0trSNTUEq z+z2cY`Up2Sv(LLZ^ZS4(;_wMH^r2Zjw)PekxD zvw^jyi&vuG#W+t-e->s{+9UT|{9Mq2w@eHTXIrPOcxA^vWE>RD=2FR)NgE5YcS}&a zMncXdSBGo8D~8SQ6Su3-`OVUhj({RE>WjSn85X_0$ThgbO2;l7g`1yo9CDjrYDb#; zdvJ5d+)=8pfv77A!79Q^B9t)e+W{Cgai2`aFyy{S60kk{F+ImI*RdnJ+dC^w6k249 zv_x?XjGAlm2fT?SV*AiTO3OA#iQ`j7f5E*yMNovt2~{ZWxVJ}HY*7W$ zj8`5#8zZvuJSGKRlSZJrKx0KAoLWLllNAVXII3|JqZZEMjlnJd@hUt(V}ZT^T)iCG z+$Ij)L~fvj86>RtT6jtrW6b4({R`>^^(9ru)pNC+2B(z|X}C#YG!G$WvK5eh91I%B zQwuLVxn@O`Wg)l*(@E+O*)gdAPwaW?NaB(~&`^hY^KWGIK0=iE7L?!!iKF~6{y3kQ zH<5Ep!cR0R*rHDIFOLk@hcNl463AG{>u9x*5KLtdGYxbGF~_~K0@;iCF-M9FTm?dt zP8dRKo2iB!n=&vVDIYivsr~SqrJ}AX<#6((KpOd!S}}X89h-ob8HGST^jZ)-p{U%e zYWu2TPrq&JQG&(?YE?P4KmD@5KHPqdL!DW}+wWj@FGuo~;i!TnLL8Ag*nXGPFcDSu zdIPJwj&obr0m{F1EyRU~FA#joo)S)ltmC}E9`?p+K#~bCHYJZS{@IY=!4?-j+wdGQ z>@E>wiDD3sdNs1~IvCol*fQ2QzL1ioNi-nZejJ_^#wf*QkP$N!(l}I;ilD<(z*_#+ zzl6|!52kDaH#6tlhN)@jSq+b@5+jL-Dwh>Nn3wqH!?Wd6%->>CWaDJw)VHuXq)eyxIJw} zb=ZoBl~k-fmiiX%=oQgciNhJT@tB%5dN|d-eJes3J0a$l9jhVrio<|J1#1xvf_fT7CnXD>ZY*!v;(%su6qcVx4Sfqb<&kW&svm3m%IZdFZk5n#q(|StlBX4s||$P9(1om zJ@RppD7%z4f=*oK=ijce;#)8W7dA@bOO{GD^6Fa$Un2b%TO^Lh9FNp0+Jshc5x5mZ z1gc!V*?HzPt~=}_g-=eomK{G*jx}lCNnLa4hI?+;wh`Fhz756Zf4kZm|I=&E+x<_A zus_v*Go{KfdKBz~-_>kuZ$@FhG6@Aw9qAc+z!AJp>>hrz%yV^Q5FBHUgzg7?1P=Agphtq*ioAluCA=9C+L&$tb0y9b@}<$GPDv5F|z+zvuCAS*?nRx-#02wj&p-&Ypv)uQqqxI zD>R-BQ>s>!M(>sL`(&kU9_tPo$s>fBy*)ivZ0n9P6&Vm}WlEYPVYPsYX`^bI@tyXC z+m=|-=~fKleC=?VKli(EAcT<$;tC#Uwog(!3`o@%n7Wpm*l_%H@R8b?8oVtxms&qK z5w!+J!B9P#J(`3QO}=u&PS_IR)zw2HhM_6~w_+KJI#`RiRj0cxs@{YVX-Jq8n1V=( zW6jA(c^g{RBCLZo@mh~Cy2-0>aXYo|P>53jEYwklG*TN21$e_NAs$2yD5l71;kqyy ze-FVWnh%bWk)m>#j(p}_>vz9*&$^6R&KNh!>!gfL{xBNhn>;g1)%9nns{c=;={wQh z9i>%WB|NUx7$H3*8|fN93~a}}C%lI~S)^7&;dtQ2d}fop@a z(b?pjyZLv2r@wwo*j_c?)7%VD{74L?%WH^iq^ zhB-UuxA=n-&?NdzD2$!@=7NR%{It%G_hVj!qhV-+%LJM()Jhh)T0#%E+(@H@e4b($x<)*70I@WvI97lp7$${2$*KmR6_pv4>FuX2XqIs0{fL+uNANH6U_6u15h zMT)Q7esjq*623Yn6+uDP_HyR;FtSWBxUac1=tdo(XE)}TnONtw-E_}p#q2)Bi22}{ zouQIw+S$tlm!eB(GmW5m)Msb?%7gx$u}07ie(9clYV5H=Cz%{{q@FtplcY-7FClb^ z7pxK$bSGFs_u!1Lf&+pA`g^)G17hqm#PE5WyMqrc$3V#DdT?uGNExCApJl zw^>j-DTx^G#?6#GJA}#q`~b$vSu@kG@z?nq{7rrlvzS?uQfE+TkQljH#^`unU{8Bv zUD10F`9sahY5tLz3$`;}Ec%h)hDfHC_KKA_RpFsyU8pY>$>!|uQ6y$UlsFEG&D$f1 z3WXF+D1r1KdnxU5+Lg4UnF}1pGULB@O)t9(ZGosz3+WM}Y<|L{x){VWjvnVJ=V0bp zFIIc?^4qFH491*JLV+P%3`z{>Z(84NBWe~=)#)MTyiXQ(qD(SNK?JOvNexU@qHmNV zkc}sK*I~6&+2kZ{y#zA4#L2F$JlPl}=wFpEbOEdC?hy;H^Ar`HC1Xql{RE}kd2UIQ zpvs($w9|jzzp9GD#Ma6o>p;fAj6)e(oyl5>4^>Gx70~rSh$%fi7Fzi@tALQC9t#Cp zo5C2n4(BnECWU2&{2|Uy9SnYmN|{O+7S1r`({%JQ!Rj!Lk>x#vz!MfNsw66iO0G~; z*hX7GlX$A{!s}O=Yut6tP%m4zo75szb_5hJevhKK5qA^OLkV$g^WGzB;Exum{hh`m z*Od9QR4SIo)I%`^5u7KjlCjALXF`)L@xYx|T4qpKK1ntch4asu(~rUlQKv8?z5xZd z^;BKWbESN6-cFAsDrm_q+X%sWnZ=>FUkL{-ifE6fJ(7wLF6pmY0iLjM%%;EE2+Uat zh}+z^g?>R@`fJQ7DvV2i4TGrE@LIO=bh}U0%V|uDr7AClS3=C|c1NovjTT^xohQ70n9z&72E z0SgI|m`Tw*ZtKx|O50qT7OeBsK}swD*L@3t)gra)Odf`{YU6PRQ>-z& zv}Y&WI>%F3MRvL=-vw=oB30^6KjFUGHUCDse~wV$g8t;^_UKETuT#6 z!XfTa>-O%)YS6gHD^96LUC2$W)Tjh(i0mbWaff3D)i7%xIQ=$L;%FQ__7i;*TKbDq z!?)3A z5Rk`NA-I}cPhf4qz(#ws1vBuV@=0RGgeCuGFWdqETBqfUze7~-L-}h zi`XiThw}ZYOYRb?VabP}V}dJusQ;D?>|*;_i00Zq4J&(iqWF!k)qv;2BmIkIOhl+i zS2dcXr07eItfsEYehP;K5hlHX^qeM%yl@lhWM}{{` zKV|&(Etv`R>fYI0PmLn40E<9$zn9EkMB@_t78GCbUi4l{j=m-N?FHc4nSRr8%dyUJ z1nY;XobX1NAPM+J>c!(*6jm&o3o2;Mk1+Mm5!^U6Z`Wu$Xmdc zluqN;JIgB;vN1R*!J8&#Uq#@S?4(8lf38A8Q1$2!5SL^x5EcsRSkifs3C~dUfMO76 z+=o4u9=x!8X;CEy_p@&KZ?Y&k7!MTPHTBpk+oNvo*U)E)sD zDrl@oehlN$PHRZ||1`L+Dbl+GenqPwI(dv~BM5uuQ5}=Qm{s^2P|%-3_mf~EybYxn zg_IHMs!Tvrdp&*A-;RJtoLC6r5yvJgN&Qi?+AF|wj_c!kEqd(}LdC?S(L%T0Jd2|I zZ_F-bjxTONdHGLFKXqPDEBe&;%r4D2(z+XU)5^tfKzS&uE$gkfPcT=!Nk@G1H&Pvl zk^1eT51c2QYqPGR)Ppon)Vw;hl@e7}wS%g#Y z%Z)}~{8P)>4YvigKSJ26--8tzq>@?wFcLhiOe*)mGK<@av0lYcfW3^&>|^#bos1?M zHuI6=<~wDjjFv5tBEQdl-hIJ+(cMR#r_OjTQg=P~JVy1Eu7WNJ3mpS_<09)pWoJVy z1dX7tAqcsKp0o4-jWyPFEK5k(VX2YAZL3hb7sahi-5F)}x&4>Ws~j6yBafJ~8NC@F zXPnEJpLP?TJFw)Lbi^wF*A;k7$0FvnnCb(m08(ZMSmHjR&K{eHYQh}gx;%>Zhh3^w z&>0g%FR+prGgr9WZkrWQ5T%vke}^B%4$6Dfed_J~i5Vf(b zDHnBx3cjm7ls9~&ROSBMN(==Q23^#c#!a2Sl$?9}+ zUiQZapL~cdG6OL$(kkP7Fy^}V>q8whs!~Hh`29x-Q*4q7oQxF`80w!F#@Kr?@!lI2 z3`-Eb?!r6%HWWWuu@o}vSlt?w$kJjTr~)TMKak9&hMPh?iR40mqBZUN@(dZHO^!^` zt03@@C6Hc|vDbZyH165j>*}ixA-B;nDe$T^mAKJjLC zUm+B`FC(#}k&rXpsR6huLjqDI zUIBBgFwC0xTF0WEHVh~{5^{9pot4*mjahMVsdD*8kzjJ%Hr=K{>QqZiM<9d{FB;=d z(w4de(q}kyEV;=K%GZ(qBn%8+ONI*6rAqdDv5bXW4h7(zrjZXMc48s8?vPI9!nt;9 z2l^m`6Mxj;)mb!#zO68u3`0q8|jqejau4O-jJbmfZ`G{>HzXT{?dM=FURp()Z0S zeP?~`DrA@!#lLCL6Z|QzdsB$QXRhu88a+9#j;U&$fG#S9jJ;O9i4G}4QK+MQ!JjIs zA3*k&&O{4V=Q!kk+_xrx_p1tY)cHu5?|3NR_cc1DYBE7#0;8UANQ@E`2=6!p@)J1~ z(``UAEp_LSz0o{M%mH?8Y)Y;b($;TbFWhwinJEP+@fYxYyond9O&YCD)=Ac8!~cCw z>Cv%CDoY`_c3B9CYJORE)oPS#+jL95v^D4MAS^&xB=9`9+yNxY@N^3ks!@8IA4L{{ z`UpkI;H1B=z8S)Fyp6}d!_Q4$Fhu#*4MjzGJN5%=_jg-E7uF^CC_;eGGRj0c$yGyR!bGGAp3 z=$N=wl>8y^fOVbh#}2=INZZD4XLqnW*ZV4}uQ7}a9H8zd!vtb-J(nVO8{JW-$ z3a(Ow2EvLIjqZGwfAGGv0QDoS3Bb}%^pr90F*ELl)TFEjUQKu)iFj|MB&|g1ESm^b zkz3o4k^Hl^9kY=`uq$T5`YSR3aNm|@AGTun=L%zQi#1f|hE58j?i*I*+rB5y4P*V4 z%B1io622Qlq~R(pm_{*aL7ETFqo6)^NQaRk)E7Jfi|nzGwRgECJVOB}^+Z*M6P6N6 zfcew2p~*DXbIJ!-Vv9VCm7kYgVcafOK*tZhc`}gJ=`^h zP!33nx+jG}^T6@YuKKF;>35XTOb=zon zKL(8~b#7nns*xoOYp9#3Q3t9+2qqLVNav`5ZObSnAzT{SCpjx@+nE?VO)v@Y_oS8bRGcAIIjeBFoswor7S)JecQnNQ#l!H27`bXVWVZ+-b=}Qi zn{=J3uL})TvMObqj-%}~<8*S)9e0Jn%{$OqCt$;=8Td?QtJ@{4qK(_ICx(GZ++EDN zl#?e+?(%ae{W35*CRAziAr0R>(Zhw1zAXlKtWAX%-TZUv)wIV9)@&)(w|6I9{zpI# z2`<9>N2{z70*bSdQ2NAsB)y7LC0&ywms~BeSW8RONaHUFpREX3P?#_kv2Z^;Jr6ft zehqS$GAB&6A=dWNQb0n@ytq#fL25>ifv{5@4&I*!ZBuP6woO*;=;XFNq##S=D5HFr z<1(u!%UaL0oRzRFQD;XLsP-nSDY`4B-8cyK}UkL`Gqk2GU5RV(|1UAY?}a4W=b!tfcK z%DvURjeoEG?ds|%n?4qXP1P!o*&7!vQ~C)L#BcpEy+HO?)vZ1ZV96XWCVpjU>bWQ7y$SZ~b z^rv?VtKDndYu#FISQO(i+o~t$NHkg%4Wd}?MdI%@8brn5?zE8*FRl7t9yoToR3Y_} zLvEai>aEn|3Z-{mzLfq7gW4M;mIJ3ZAU0TWB&1JqApOwj_cx8-rfBFRqg4GBNcNHQ zNTVih73H7gLs^rL^hmlvweP<{VeeM!^|T|cZylP|3=xNcJ#vL2GER`-hXMTNx2GZT z1wotrE5DVVe-8!5v5__N`;U}QVE^>s93T3}d@TWsJH+@~CZM26{DRBY znH!g31KV8U_mg^0d9i;-zi**s##j4Qj2`^=@ekQohaR$OnPB|;`>m^<_AB52JH03H z6?^kkwFPYq;u8kh#@W|tggwbs|Fb549!8A4h)1ZoL7Bh(bU%YGAin;z*tmG5fa08w zoT_cHPC3%dV%2{h@re0mN9+CrSg6!o@G3j_`C|uq0-Ep89DT%FsLu{u{M?KEhJ1}0 zKEIj!uT2nT9%r>pO)U*d06$@$wxt^ zpW~rCrq%V%_#^QDvi2X)ab)M6D1NG{L)F&-jY4jMWFrTLA&2B-Nt~56)=2ZW$JT#r z$@{#fv};+bv$n>ox9>ggzJ0E4WDp>lAOM27zzBi~%z0Q~PD6qL%%Q;yMlj3}9CC7eS=nig^n@2 z2r4uI2jxQ2{Nx46n$U}D9m)OWk^}pNIu7|lBYH_ z_U2qft$H#At@XrgojkP*bN+1>t6l8i*-$dJDc;~`mr)nG_TZKtBfzDITjkaIK}GW! z2CJWn6?HQ?%Y49y8#g6kjF?jM=r!HSDFD8Woz6xd)IzpBU3pNXwd5qoy~M@#N<3TB zu%IS+DZ_y6;o}DMT$$jAAQvRrD5i;NmT|#5nng!s-jqN(zM`&sn)qg3nU-^ra+Lu) znNMeLxY@tcN1zSu=Gy#WPkr-;*nw>}kvj3d)FNVtaLm?-h~7f{!{f842om#)GCy?< zbb=mP*d@H8`m3ao>X(;ro-oMAX~wJmu4B6!=LM*yF(EQ(_j(QJZ((G z>s2=+_sYd+VtED;$9ZL-2k|o?Jf588S%ancCKiqr5VCR^{^cS{wBM5$4oli zX9M@kU;T|@(YStMq*cVUS$U9*ng?~aEu%UfO~G#;?Pk15wVarOu8vP&B4H$x7^qFE z35)=AzJA+v1m{LONK@@OyMP*OZD6!U&HBh8ozS`vQm_dMIzE41yBF~eWFBntGacKA z1RV0f_ay5PeJ7Q6x&1%PF;@Mu_W$ht@AN8 z^5Xeq|3D9VO>Tl=V9s|g53a=z=|V`+m@T1<1x7;bZn;o9ZB0 zT1H-Q<6%Uz(=uLImvM=_`1&qTWiLfQnArbnw)d5P6p{BugfgG-{bAai?{UX~0JvY` z!EW5Z7OH!ih^fic%u^joo?rw_<1enRI#hz!yawomPdvXd2_qw&OZZSTnQzUabookeZuLJE`3luA=ePRbu$4{rk=*k!z^o{uirX zOK)v&L$k96xmwt5H!9>rUq`d()1X|~8_eq#-8JHDAE zpx>Kf>FYDCy$79>|J5QL*h$}9(1jqLe@`w?t$P!7`SBO(JNPrJ&|%UCw+K?6rs+lG zTzw}a%!xP%XFDw7_b*q++<5`3Gg?3#WFvr8()NgSG^ zM6U!N3w{Ipadp7bHjNUeN|*|>2vWZGz4YoHL#03qH3H`4qOsjx_eHQ1P4*moKKTCm zzg}iFmAW219j<{abmwdy`1Ud+)3yR;iwLs}#BJL??|17G#t93$ks2tO3v6jRP5I{5 z{vI!N;?qc?oYl1qroL94X|?weip=Arr!{G)nY+-zL=fLdc68Q60$R}H#zOW!E3vuW zlP$B-y&?HomJ!)SOZpFhXDPch2`iFF30yIx9dg`BI_mhEl;N!Ife5;NGA+`D=TD>V zphXat7r{{)bqAHd6J{gkwoEWo5Q!oFW*+bhZ8~}YF(4TW{%^9(x5KLYR0OSZS<#oF z%|13|(g(n^!ErXD&HXv9yrwj50Ks~*a1qqm*BG;7whNC@9q^d8XEW+~GdA7!@BF0r1hN@n!Mxdm2WUXxD3m|z_lFkIZUU0NFsF!G%ilKr} zJWoB(Jadz7`Q|0vfYwyyd?K01V=U!=x^p`pd@=7!d^H_lddd$l+WN@M*NJqg9}ccV zfOX_VVn!)s&P?6yHYD_sFmascq09lbgct(n(xpFr%bmXUAO&0J_#LQAguAY2)aFOf1zxpzXh7m`+x2^{Z1K?8RqbIi~g?IaC9&p<)K zEm&{cQWilxWQ*$gSaRR>>lklr2#WZBDVq_%OYJntZW!rKsMLo8e4B-C+y>!?1Hp)s zunsV^_phB?aRVtSfzN~F)yxd)oE=XuBpt%uAWZ%GiILww-?bI9vmZZX$OPMzQTvUE zm1h$mFqNre>KQ9%3sQ@l3rP=s#A|8=XMdGwR|~o~)BY@>g(~%1@bYgD9qg`iF}3Ij zo@}Bb)hH=l#(;ITIlJC1G zd&`&34r2I^rPmL8<|$%0NRy47?AG@1?MR%RsrZCnRkE7x!@F3EE%)0a_+9($zvr*p22*8y_UOl%WelkAree3?*6=^zP%V7 z96AG8ormcxSEO34FFuc+G)QfS3YZ`6#OHWFiW*ALBw48#x?hrJ&{z@XODvsw}yy_BKXV`52MkfsvuV-U2=GoY%(ut5P|KgNNw^x}#q4S{`b=-N%eo(4IP!Cr(99 z!_f}h2#3;%I*TD)k(`o!C^6hNP8cLRTMG#l!AZ>qh}PoKrTsjMcM7baI(!N?{)2>z zaIWleAXIhjV;p5y$t&M5)9sD+m`j$HgEbk()NnBcSAxnj5|1bZeujRmXOzv&-?P17 zztfhmLwi_hf?t6%I~O-*7ydY>8*7r`>(kN96u|;I=E>ScCiz%w z9lNW%2Ahc#8pbGaCs)&vMaadHR;-8q!l?h&lLbauWE`*?oehf*+Rh#}t%}a2Wn;ngoo#m1<{8r`e?2JuTFbiq!?86QRZyyk6=_HgFn+$d{?Ld&0GBnb^-bw%l`V|$;xEq z1zFJ-Xj$CV<1ZXP=tYaez$D+v-8->OMJ%N+j9=4!1FOUS1h>6WcpVOEPW#`fzvfrQ zFIiOcx%!enjd@{^w&M=w>B_%RUzDMJkW7ytLB6-OqZKQRJa4{)*+Dc3Ln)Yw0OOHKjcK@O{^|? zjV=39Ks#X?~;d)gUv z3w2WvKSO>}Xs;ghcFjR2x{Lhh8AZF-B5;L^?hc)5oXBw-0q`R0yR1j_%Pf}Tx$E1h z_Sy(G0?SREP-4EHb&gba`F*;M?yZ3UqxsYcZrQ zbQy?MxQYlZRiH12;>Qo2Sw{8@rlgo1*}=sEjd!|w$l)@zB(#*qP=Ep>8)$C9dRnSd zt60$(ELdG2G^1_g;Io0NNO)v6Kcl20kfWaQA78yP=}8qW3q9|1WGEGu<6Q=V9e{R zHJNYVzDl%2W8%2cV%AV{J?6xVi*}6VNXFX{o?JhG(xrAq3VMfLZL@A#8U%M|jf4=H zccDOq^{w?M#CU!LxRhu~`E4a{JiDF3@XEJfgi`t+$=H{K7rF`uLOcG+1K;c9?|qsb zp(864L8=BPS8MrWpDixC4rq&6PLfH3k_3h{Pv4VJUY|(w#gGvTobO8_X(!G@`CHV- z{bjU~(*z#u!I2vYZZ4|q>z#8NL1+S|6?Ayfz`B;!LU89F&c9h3ye3F@Fk=Baz5RRJ!! zzg@*_-nIQxbbgjV`e4?eLvXh6Gx=5?r{~lPrR_n{jyqNS$(u`fij1AR*jB1-^8h-| z>~Erdx=F9Q-fzR6oe6w=iEnD}V$g>*90+`q*|h-GHqj#1dm>XS$`1+lZZs&ANqqNh z)7h9cm!P#wbJwYxQ`W5aIElP>u0GJdYfil;`I250G0qfUDiYh~2u2+aF2Nu4UrmZ6 z&9%!9gg&UQgmDR2NLLAtbL9~R8;K;>Idp{G1N4Fw_ZUoiKi1Mqn5z>ULY;suHp6i` znLZZ)_Z~N%DO)KpLZzVTd(1eX%l|Ucfb>#~!p3TL8%GIF6F#^T6cI^djMW@n-=&q9 z6PDL7k9zmeni2U4Cz(;Mc#ZUC4_5ADQPB|~ua;}nhCLhcIK@2NNKN4jwfrCbs3w%> zL_-M&kEB0cbS*{s!(SSj6FS(7J^xMp)1K&S_BjZqgYygsm6Orcmza}7OIGOU<{Yd^ zG)AxZ(BlAh+$N$i-b-D*wUpL)L>Lhj?^ahMqGC0z+`=k98tXC$-vctJlnpjcLD+LX z@G~--GMh77GG}JCX3ol#b!F_C4ysc<-^M9KOQSSe-AXF}9{IHR(&rboBB^xzbx4Y zz8A=pJ^Odd{{rjXvcOU9ukcs;^~37~1!ynrN2cwjaWFoq);ELgO2w)XB9?k2Nf2H2 zm!VzAE8H1fyI=9>(610cL|Xqqv_gByS{0(-v&xU^u1Dj7g!~e%JcJ##8B>3OAs0oE zx-6|dZF$;?w3TVA(pIObVj3YJ1O<&lzy91uk{KCCy5^YQ8pP}jhv5L%-J{GMG~Sl| zR`vGW=jeH>4>1~F-PB{Y<9dXHojSSlg=9ydY99F1i(aitU$aNfqN+5D{}GHi>BBvl zha6-RN8M=1(y4XOcD~P@892t`w~&6t=N-7Z!j8$c4Bs~?DH+;nF-)&HaURL$>_g@D zlA$m;;8kD{RdkbD*)3r{lPVs*b1vOoq5ex)u7-A`Rr&h>tAW|7|4W!3!89Es!QGpR zJ3N;O?oK2a_mO#k-)gZ6-CVTD#2VmRX2IIWcI?V5gw+1DNA&4rMG-7<@`RcMj!I0J zN!(p%-L(gc4oiyOuUnX{Of0+ad*(HBoa(0H_|X(eWf2K#o+}P4n6fQ&)N|c^HWJ(~ z?Z;flU0dzj?Az@;8~V}Wqf0z!iZmMhxZN)4Y1qg`knx;sW?T(5u#>~8M3m!ABzl;# zySM)~*p6gicK&S$Jcj`wSvndoSQ3jhFl#>O#Y9O4PUW|~`h*BwhEWc9+k9G@6KR!_ z1Kw)?Q{F1fSpmQ?Aj^6GI6!|1G;M4 z%G_eV@(_FlHZ)m40GmwOFv=R3Skd+|W=rx`-0MZ>%Z+Ac|7HGBwVM2lp9q1gRD`n3 zBImZP{A~Bx-TxIqzPXT8bTbw6OtgIZ?9$R5n-92AjUeaoMR$Iq=b`P9jc{xT3G~6B zi28yN0Cj$3guI{;u*xPq=3A^>$5u=W0F{$-iBd7f($QiWOp)Gam^ zvTs>xTQ@j_+R5lUO0clM$1mJ`OFfEZd%8XBzGMYUCY`^H;q+5IU4Hdz{~yun8#(2w zMQ<5>qvkq=ioOIfxLZkIt=s27Z_sQ?t;p-K0#9PmD5 z7Wo$Y7Q+&V8I+M>P`L(pm%+&Dga;cgkr?(Q$PN3Op=AiFL5+~o8HykWmH>ES-L1;Z z-{9{|K* zdpmx($0wCW_>azp`E3|+kCxaAj*E^<4nxmiZrTCcnrR2dpe;k>%O0f5W=%#tQ7L3x z;}1KIIP?l{9gQ~!Jk$Lej={-Ul+b`20%+ck+ddJldLKZEesX4d6JGkqc9oAQ_fjNd zpY4Dhdlq3P&_hU8r~Yqcmk_UjRYT`2061sKzK2iH_wPml$%)t%a*vPNI;W|BQl$7X zkhRC#XKS*+3? zV`P-%Kr=>Ld@gne;3Sj_`| zpF>a9&}z_%P4%A+>h8nQ7_+g&9CzJMU%1hQ`Hc=?7)K-;6VYx|P7R4L$CIWf7FrsGHs2>9F;$P?1WJCJjg% z*j2Ws|Ig@mZI*R6K<`|&jKRJY3%7=C*+Pwg>0dBKFneIEggieYv{fXK z=swgPvZ1|VDCjguLD2EX*(=nZ6T2x5_PW8*q1fF|X+1^_SVO<-&oc@7R9ieMtk0^px}2fTN+ zO(Z+tD>Moi_2MnLL8x|^G@be>bP=RB`{yO=KB}&QkC|YuV+`!|xLFci_PylB!I*Td z7oWPs33g&Oz*08sR!~_&hm)XZMQhTxGnZiPR4#ZiV=?*DG)o{fOMpIi0q|FCTbWa~ zh;0>ppT59cWTMOW98_$QFgj@|q|Y%KaZTS&3-xNjCOZeE_c;tD`D9Q^p|StYGHUDT zCw7dsAInN}T>x>~`hP{=3WbJ+y zwJXNg)E=s_cA@$%e$VFv|G=eJw_RbS@Dwzy$%kZ>Lu9U;FEmunv*lDzZ8bhrHY-xA z1iE@!1oK53u8l^i%&dwCn8n%z2s~t-c)rbio;i?-vPF(-lCI}U+2S;rs2u|^h8SD) z4zIi@V&dCeNKy`I^)&eOX?MR{edg&_XY^%yaB7`RO9%*hNQ_5ke4h(RYqE5C)moaS zNwO@-t{xA=Vm9{Ed zwQZ=Jm07cYaA%EJh|{W!(O%M;=v~NvNI#<6oR2Z^-pF6&J8t{*g$%l9K7isGTr6~M zNCGc8Da--yJz6swV*C7$x~j1LKNq~sWTZ6J5e!k-gUg6{Y3qx=@i1Zozmea>Z|1k~ zTlsDLh2)FL=cx14smzO1U*_FRLy${HS71XvA&ed`%Vz`khcADpMjH^mp#dU;fs^ht zZ70-YUu2BrY@Pp23P-S^j2Rp3@?APohvZ1|X}X`VQhMolkXSp@w$vu;KzX{>hCY5&&#bf0-fuCd7tBw3K|&%I8G# z$>g<8?7@+YzI@=j__eN8?)9z>F8oy*_@GoXns}GNw))w{wBF8fjAbSPTOR0Y7cuo&E_gfA&w3BK&_MZ7 z4Rx*}f)OE$AQg}PKucnEkF7lzp)h@3swSpf{oXIxjxXHDn4nFX{o-t-1_q}foab$9 z6scimVJ_2j&+H~UR+x=}Y!U^yCKIKVzoB*0Bpfc4jeF!eC6x;p-=s(nktz3rHnm6x z2y2Z(nwZ`}5rbI%9DZ<|>7%-t`_u#KA@ztl!91m&QEycvfKFvet^haBGuicOLiz|15tveR}$- z%!^4vdN7^mE_b+E%S0Q_=p*Iex&+LT;Htw<#;eTW;Z+>Q04{>mSj;~O{g8ngIR~o2(}7`yrC-vn^dlbJj=(~MwcNr=nVRV^2w-v zE`*f!R0HSnkdWGhYAjZwh7 zsk|jIr;HPZ>7iPNNIvjQZvCev++W5GIcjtUnX1ky=sCTgxRB_qUuY1uG$gb3mE_G7 zL7HZ9tFLr?+#`If9a~vlrt8O4-a^jrZc;p}ELWg?$BL4g+413kdM2L;x@T_ucdW0!o@g%Nb~!ME04r!}7E z$|WjNiRKfOahbQ5y0EeW7sii=5!wZ#{}M^_hb(Z+_Akq<^hb-Q)NTZh%Yz6!&RaQ1 zm-50i0rMcVOyee#PfE-d8#|Y}{LNeyfr+t9)@Ewieg<7ATbH#vyvTtzO(97GQx@L& z6tQeFYLfE}+tqnYMOR3DI-eWcY?M|&Fqm#uT?nb$l44horsE@jo5kr+HcqHR?Wj$A1!?<#f~gqL**DfW z)?D9@C3xn5R}oCl^~XJ2?6WI+F^lcd@}z7UO?pbtq+1OLje6@~x_2@8acq~_J+SNz^!)+(X8xgBLjqc>U?+%=# zup#7FjQiCKI_{VA-N9qaC`?Wi0C)85sIv{OMx=#C35!T6GVT9}h+b@aYbs0<&>25; zKIGp%-nG_-eJb-WjkKm7Yn{T*ir&hD0)}FY0C%cJ}W0&@KrxVJ(I97qszy)*0X# za#z$E|NfbeHHC?R?X>ojdG8EdvOT}oa02A+NvWMmAYEaX>Jp^fWt|dM#m7qMX-?e; zi@X{N4EujwcHk(h+G`nKeshTvEtz6Sf8dzoJa4;TBP8VH$}I$bfxCfSywOaVeOkvCE{dm!RFlGL{8aGC%nFP7>5}6om6 zKJrx?#t}1hf4>6NxA)i(m~GDIaN0Y&E}_ZC2UaijeD}>2G{_ul_0qJKzU3%5r6BrF z6~>)*fOmT0)HZZ{d!v0czEw4WY=l3~D2sqIXZrG$?TA*EVjR||*1J@*G#L%CKY)@; zcFa51a>z-LUO$AweFO9s7E~t91Y$bfD;>-&YB2jHy2ZxYb?*u(N!T*LMqa&%vX+O)t9qz@#J!rrr?B(=29v_%J=_Tg z0k~ft6RAd?8w~#zWOuO^5`G=0X(YIJ`>+f1Hd&->*UP#XqDgN0)` zvu=0U(TQVq>X9d8F{IU|KBAAYx<^drRlSp7yg3=HIM&*2$qAQd%tno|frBFZ?Z!_r z^t{A8T8)H6<1g(oAZa_Ox<{X=zx5*n=SVGek~}>MAvHF5$<6#uqh|}Y>jQ=oqO4SG{|PmXKXFDjXQfa z{`6^E9J`%+oI9wU)I868&jOEvJ5CmxP!nd8jX&Z?d9SC%iw$*k$GM4l8P{@v5o%Nj zF_uq;n;M!a5N4!Zxrk8LLOut4nz(NRnnkhM6vydZy6ZVALz4MXkhO)`${gnobC|SW zsjY*P1@(NEnX_rvc%y7g(n;)H$bX*lh+anP5oOIrW>@A`^+hncplSW0yJ_+aT+-Mk zE;LubqA(238S+H_1STAF5LplZ?#Dxioz>0w@Zg2W1hsY@25`y~b}P?AJLxw{fcvNR zecKoeOACPaH4FjuLeg+}PAt7!E1dww)zAsCo6>erF5 z--UdV<1jUCExV3%J=Mk6z6<$!-x`CVHaiHN&)XP$HpWLgck(76Je5We*19Coc_g@> zJMn8E6cDNfY{roZjz#`*PlZP})|1Uxe`An6LVEv~nlkOVIqyZIALgLkXyT+E*lUq(*J~^TT1TcR(j8OBKAWT@}G6|vxe{O7+WMC_$|q|xZ9le zb%baj50V$VNUvcMnVSHClfI#X07q70hqG;j^aJBUgcO{CF7o?tAZBMmRZdf7Y)mj^wse)|Oqd8`%sxbGlX> zr?H`G4kY0jIWiuW(tPm87?0QJUiho4<(;&K>uczeu!CDNc#T28Z=OMfT{X3=N3>yg z%#UF7L9Us3rE%S~S`RNf{ytnKq^RBDq=eQ3`|P7>7ixL*R6c-0gSOCcyUZA{FK=%A z1-irEAggD~B6~|?|EK6@GvXdjx>U>zc5dt+NKFKGb*xAs4d%Go8#&mMnP?RBN8nU1 z_GGVKw7CH*aRT7pk+w7KDZ~{(Tx~lgbcl9zMp{IolIjFgR`!#mtM~R@L6x4D{}{%I z&NndwqdA+UrwrtCK3AA;^+ zHznK=euov3IhXAb3V-_gA9L;8J)a6j|AR`^_aJYJw~DD|eop-CTiPF;eeteSm>n8G zN8bBT*y8-jCwJ|~n6vIbxBP-`bd)>`sadg^`~=9^;l{H-jX99o?8zMPF7|KLXm09- zVVz|Whk7-tkyF>;mDh~tSQz8L_S)kRrE)|%oCkLQ3_F&J%(4qfns+gzp0gPYlDLXa zN|+ka0&I=&h?X634kZ!hxP+a{6cIR+UlMyfKj|(DE=P)QH%(ImXhC$jLA0 z3FAQW$1pbJdZV~Ho#yrpT(sM5_jk>7VjuDSgzdZ5*>A7;p2dFDbWpYElMUDRxDXaJ z)Ohc;^T^;*^#J|CoF01E7MwO2?>0@@0QSMDdDzT8rtZXMx=7urwFLVyjKl1O5C$$wkC6s^ifVw4)MkCrI;w51{xA8@n`ymV^;1mPtcb)UFpB=2P`g z`*zS8R?QOs$z;ug<5u0LA~ouEm6fNd>pceeELSfw!fZ9YNd9u%F%hI9czuu)UeSw% zY8ODta!-RkSBIFsjTa7lD~E}xrk%-LI)EmlW2^Gtcs8{U8Y`yG!oomriOk= z3@8ZTTd^mq6fz@%%6#@kw|I;JN7iV`-!J;l*v5@HV#r5F8YvE0{Nsu~5zB8ch+Axu z^L1h&5s9suah?D6j!Dv;#%%hH%e0+#|OZB zH!I><<2j5uoNBZfI(JRo@jDT6U%7__3YEH16i|l}itR|7vh% z9gSVKy2YS0v`t4IZiFv2v@i9aoSJNYd!+h7u2WIjZ|vn@F_;2o#r$Yj*uzNC6K;` zHMCY>H>C#`A=wBZZe_bUSBb4jS=V{ZhEOy`q<-T7S5sCtN;By&p(nCOxvic1?WnBK zS^X>?F>O|x=4#~;Ll!}>uC}v?=PB{=vw*DcWc=_<+ThGbZd6~5fnlBAtL`&!7FHzb z^(>@xw+K?3?Jf41c2XTnmRu7dr_UY>s`<#gR=?lzgZ6SYfFv`{EYOU4&|4uk))0P6 ze?Du~E+;lv&H?W-zvhJ3jX+8u{VQXZBwIE`sFX1O+afb-AcKj?CdXzXHA`#umk<2I zR0Pp08*=K0ThtBNO=Ngna^F1hR_b>N%_+w&&gGuwwG&DO3qdt8XnPAr;63PO*aG*`d)@7(!d;6YvUZg9B)wukDlELw zQ{u2|GpmbXC5@nlju~j8XMKGOup#9SVN?q_Rgl;T&q(Ms+<-G7mW~xAh1}UZoC26_KTu?ne;y3r(&Ia=WSSCn(2xa}OJ- zQT^tp7@3zj|NGt*&$85Sb(msXi1h;KUYvCI(Rvm${EmlrX{I*BmmXRFUiC^bPo0U5 zjR9(S4~YA)%w9-!FN)X@eZ;_1e9=EO_sq;kr+|gGX8Nt#waDG*6RVf<{XGNd&VSG9 zr74Y<22eElXv^PvvL%eL%4~B+^ex-m*6SGVs0Ug}!$O&B$rnZ^-W=cuZslTr6z zS;UhPJ0r({b9|^Hf(3iz85ANqGk`+i*V@M&+F7ICkrBp9aCMN`*RB?P!}x&C&52B_$Oibs zm*`&!`ojoKWI|iuG9rG}_zv`;2DmW# z5=E+_l>emuFdz6CdhbUP5z95ON-Yy?VMNOrhxehHa*C@I+)_|LTA+!EalG6WyCM|c z%y^G`iLQ4+>x4~qHlL)I-LZ#!XMw0EC{8&80TCrVyb8CN((Y17~0cij+ z0$^zdnnF~EX<@pE$ug&U*?3#WFtMzlix6nqMxQ>E~Yae6H ziaC#j>0S3NS|lm z$_L-Y^t)~?1iiYM5|Mq9jJ3-e@N@RvwmmkZ?V1IS)p0|D2H}ST!H5$>qD{T%-kr82 zfVT8|P-qkuWP~&l^65PAUCttRI~jdwiimG!&u&<*-fvU1c$ubW;%Wp(q*86frttZY ze8PP=CFIm}VkGsc2vT3OMssrV`HqInHKs1et=nw}XR5~W^2y}e%5Gz~vpd*Mf1_Qd zfDmla5(~MN3in+#`CS` zCSbNox|mWO`d2YooFBo2tBLVdH+yfNyye6EV$AM!u3T`O3yL(RU`;(_kM*e9g+?$e z5{xL>?Vr2$AfWGdIB=Vt>Qk@jq=gv~>`XGRE7>w;m()e!t(Td%5Id88UAy0b4Lkhc z2=k}7XmKz#LJKWu+tVuCXeeYNfO9L%pG0O#onZw$MkBIB@(A5IKVKdnvOm3Zthg zTb@=xgXYpDb6rhAwqw+85gBHDu7JbZ0Aoaje8!fq60!S!=!D>91&=R38G*iyv6hBH z++5Ks%@w_PwU9xmI?#*zH&Lx2^*j5LpA2KP8Sg_99!RxQf+{E^rDocQTyXrQ;I zhmxu!T5|vx+uLZzd8EBfiD|e$dzO0M--a2UOn`vibC-mv&!oQEe8Yi5i9_-Sy${#+ z+6G#~7qx;^L+258Tm1}HjYXE@woO%Agx!=WVdl|0-D-gqk@Li6kT?^;`pAQ1^{pY2 z4881_S`Z5Pit^mZnv*-%L3f8D)SEh@lc#qsByDkQb!>BNckFP)U$bji)6OpS2v1XR zCnIT<)A@MX_Y~%Zmk-I$biJ^fLt8B)@2&Kko&)8=($Zcn5z^WTT1|P+;FH)44jCDOAlnF>}G72)Ig~;5b zTfTWoH$YpA-1sxGOu&kfQL%d|`p0|H$V;}y-tt7EG$qc4Gy`;9cP`1P3hT6~3Z!;s zG-OG3)7)9oXm&gV+Fi=PoUHNGOzkgP)pAf4P~)n8?fiqV`5VgQo;-pq9HKv2Hn7Kz zjkztZ-ipAU9T7~RTV&#m#h&|_REzpnxk#Dv>)$-E8x=GT?!n7Hs!gOjdPc=aZX_i^ zUM z_wCWvUuVut>QJ)K)YSQPmZZ)_WYafehS;Qz>Iw>z?c{*B-l09X>m3KG_F_<&;T7(D zyyd780i8!xa zMTr3K@E<_YaW=707kkj#H3!wGt~i@+9iwxr(T)X+&2pvsY#sai@S(%IP_V(j(f<{- zi)wVta5OoZ9W9QTj#kGk$7~0l(npSuEl#dgUz{sf-*mr4rKr77NS9aRpn6Ue+fZ>Z zhR-*Ir%3-B%K<}E$%F6`Oba($TR}L+x2*UWGpQ^DUXhyNYjHU>^Tje6D>TNCDZPZg zORx#$0fcxKkydib_fj!gm?dEe#kr7-GM`2mJGv^O(go{jsY<}8`y;`9fW=SXsgf&F z5wQ<7SkTq+qpb{k!}RfvE*8^B<;86Lwfy6iJ#+-k9>ystV_GtgdZ4VJf_3WGmC1hq z#kc81X_5pVEdcJ9_E%^U$(>$J?LMh`UrDmbNS(pBsu67eTUsg z+(+Fi+Dc;8dHS}UJ6KIgRqFNGPw|=Hm=3YkL>M@a)tORjOOSd0d3QG=qBx@mfT7c# z9?#4L-<;GK>Z#_NJA;`*gT_ zou%(HnN417;x5JYbuhvX0Z=eihdf^K$-NXmpZuakm}whhXY>n#nO}3 z`!@J^?!W+BG5=Z~T9dip3p+_Kz{cEu6`<{>__2^3;d(h!CmGU^sSr{y@Co;A(~|WH zjVC9%VX8R=QwR4LRh`-EI?tj4$jOfZzp@RlruA{J?Hw%My`6J!nRgBA1BU}FWoyzD zMdaP1%pJ56KDWLS>$$4tyVxM$LnXy10sk?st7k-awMytTag}IDU4H_>PUEz2t`LpvJb( zlq!4AyWn8hg%vA=nz@V7Q@WI65-)v4@#6xfizYY%guEY&)2n)Gg|^ z8pdIe!gr4zu$jHQp894Lk_$=>{#__sYX8%pe)=;bRix%;og04Jt$vQrqsw0B-L3<+ zvcPdGIks&ckDvJ}h@}DLfJe@7MLd-jk>1^pGREu%xki`SdABMsE^G;;Y2F0-5^p}B zrRl28bR^~?o=T2rD@8`C6fp|o&)|KOj1dcA>8Bo3PvZYo;#iDFXGAo#EGjV|QIkd+ zZzmAC;_Rb(j;poLB&~0yW3m{o= zlO7mcg+PTHRA?%eaio74i2#@dES3v&b%ltT9v1+AfzwY@4BY)=ji2@$D04RUM<`7f z*`>GqtDGXVqVwOzkB7j3eF(&t__k$}>Y2V)UyOPEjQfWBmeVq>$y?!lD4L&O=TC-j zjD=wZg+;|Br3)N}1K(?o`K>`(Q-hn2yVD0NevC~=hfWykvkOxvBNXCo2sGJ;5U!FP zxW##V*L&l^Ar?OnOkG^L1+on*eao)(me;aD zf3xZgN?7cmtSa`ij;b9Z7TZ&-c%B_$kFu9M#3h3_A8*6RleVqQA=yO?zYVahNczSV zo7rUR5`L}2P@I9fJr#D0A#5@BBE+FXokGM~$_HL>eGi*i?V2Ejm!lwalmDUjk=IBB zS0*?js5}ptF_y}*gLBczz(z0e*}uh47|I!W zYLOE1UHpAl+`Lii()SMJZS{YX^lj4fB+D^yqmVC^tJ%?k=sAU1y%>)9%K3_er~&oy zpTaxGo!!n}=8fa{bvE2Xtp=TWfSJ`U=`xrPJZ=PJA~q_76fD9SJz0_=SYV>HCe#xj zL8*alj_I*U407iM!RpM&muDoCqQ7Ee<+i5a?F+szHW~~^idAC z`&>5)IAi#c;63FzsY%?^UW5Vg?qRj+P-1+o6^Ax*5tRJJM9a(V{vH0-B%Zt6VHYX| z#3>tXE!!c^OAE|rb%D?v!GNT9AoqZCXzQbLffi6`plpCsNc^1Xo+7>i^CH)XSV5$< z7-~;RxID%;!W`VxqklRzcxyeQiS+QaWoc$-{M5pq>7P(rnU zZPz%8UGG}sQhm)anzJn!o~*$gbV+~CwEDTd6c!_#1HRX;L6@fMAr7WoNZN=+UB!l$ z8#|Wb^Rpkpn0;^LRYii(AHk@Lu1l`Vu9wULN~GF`m>3ZgcO9SEQF}KRlb4PJ&$QH9 z_lgwt0}vHt0KAx`h}^2`-41oEftX<;kxZEb-jiA>t4;?z+QONH3k!5mQoT9_Tl5$1 zjML{8O!{hvCsL-)mB&Rl-Fk<{Nly6qYAne?{w0$wGEN99z#K@5DJl-Q4!YuB);zDI zauy_8^xOXJKb+=zI7L)JkkNI~)V1F42Nu~89;bT&nM$FTsp)|1;Q+t=2UWSjHh7ku z+H1$JaD&h&b~U~1#5IfsRstCpe8#F6vc((@@HM{xj`6!Q_GIYiONGG4xG(dPZoptt zoF~j3oJUEO0;Y2)Hid1S$FjOB)Ehk(RGzjg4JWEs2_puC2=Y{kzJzchs}WK-LPQa5-yL9f6j;qn4V#xmWNx3`bE zy!5FZvsfuH4awA{-wg-o@J1KJ^t8q_o_(Gxg(XbqYXRUeW$qZ4t4gqUS40@hoGwLSy+6ph z?^0)vn{Lsqi0t>doFC&u2Gnd&q9W?eJemI!7>_f`6HF;40f$i*PQ$v4_+0SKNg@jx zqUAH?3^nmj4p-ktG(t~&lwoWsz;3Z^-g+Hd69vFWl53NREJB^RgL%RBY6~4{l`td! zVZa`;f8~r>0<8Q(KpZ-W;(rR0r1Y67z3esiI(vhC12+Tm`1$~f&Uq8$jf97nZI&?d znB{2Q;ktMI4;U%!ui?ir{PZq$kGfA)+N zzK6mLwk_?Z3B7#m(trz74@sWGIc zvJ15u?n#i_k^13BLttdw(nczRsO?|!+1N_bhq8?Z{~m9XC-$!U?T_smH!repdk+eB z`7f~vEVMF#t}ct9cTr>#YrJc;fjgv$%`+8mHg}mpFRy)Hg#XE zIJN2wH&ZYRN^}R735_8C=>1Y_;J05~jBu)#7 z!c%H~51RIWVI3W}31;>+sKTEwEiyFcA7c8ORWY$F7?GLr8#$etUYBW|r{^EC`>DZ( zI@IOKxWV0|dr4QQuc>1cl57$@u0lwC$UN~p^~BUkt*Pv(QDH4KO?9E`wqqG~3`q zC{e^R2?W^?N?T*5V*1Qu?I^hF9i>Xq$dOdR$5vNpk%(DF{RG}7903zD>kdh@h~SyY z(N4k0RseMh6@py~Ni;s$KF&R=2^lBUB`GFdOahT=4Gp@9|@CYcO4?qw0dH41`T!>PBQ1l|shXV4{dv z_{ZQXMA48b*=Xvn;7SCi|7{~HS)z_C^OdcQV9RniK!d#%X2EQj13Wu-f;d&eOd;~X z_ril24j5XU<>(jzhirothj3PMK<6|-zWI1!93FtDEP)%;rBmksn@9}@*y~5fH`yG= zRq6e1T^m>ykXq5FmkC%F@qTQmSg1vN*rqnpQ~POQX2gyo`o0)F8}BIR*`8lOw^#?I zI;Ua;E%{s_NN%GnEQr`ulcXvU(}w5_Oz%5)UAu-rSA{99iQ+@|ll1RH`vr_w}bjYjEIgVlO$62IRacw zZfxVfzw6v#oc3kzTHu~}asjHia-+4>1D8P%oipeNwobv>VaA44&NHQ*Wm8aHVSKL) z@0flNv3~+e79<{|iM;*@VSxN}XZl)w-?(G6g|L0PJr0Diq0fa15o>|-z_-sol>cVF z5gT2{0?UWwz05v_K&b!W-T{flfbLAN&-PZh{x#Z6B=rX)XdaQSL`x8XjvpnQf#JH@ z7@~G9xx1Lgzh48n2z~M4*R(1ZSt>(251IM=0$${o!r5e=dvcVMs?@RY=lp0Z=p#(k z-(9FKC)qV14Ugo(m@rc#c4;a)C#dLUOqvN3U}-2-Xc94$DbMH={i=RrhZFmnOU(ym z{q&u(y9{;+<$-U84pN!tT+(sNhB_isGpx1=<=p}nQTRhB)hc?e-EFtep%$+WPCzaA zD9GIJkI{lJ`>yzoYhcwjDJK$+_|%nIU_2ovW3xv=rb0E=|4z~&I~TmCd|FSKa*>wG zMQlh|2q|S5*yLY<M=KTpxFrJChXijIURvWOJUZ#UrpMz{t7md+%Gi zs5Y;ps#f*_S_*Fv5A#YG85#)wP;)`)myvsB-9@*uF{BC^CqLl7$)O}AbWsp4EN_s7 zaGqVTztV$UNak^ZSN>6~kg&^RlzD-~)d-AKk31JBqsWZI%&LW57{x$`>CX7u;7WBl z#~e|~5eTV>EH?9q?fQ7Irwf;hzah6^T~W7M1$4{S0;Xn9WWK49g+$u%*00oWUHJ3p zMO^82Z(=qxWUHc648vD`oCoqZ7j4?i(|Y zp_oVlbAJugF0TE9u^8aZRy!fXu@hTHxdYLzL)7LqT zR3V)M?JlJYSfwcXM6#q9`=6^I!{<4J!3ey9^ifq zLFz#G*XaEj9Y*O}D()k>^)NmQoJaiXzSBf36JsY0uan(FHP$XvKi=<+NQ}xRZKLYf ztN8r{rKbK1LnR_%hi(-+{Kye(o`Txvk6_fLv^8E0s$+ow5IN9^q$ysJVEOVzZy_Q; zKKPFMFW4^H<~nawOZmZTy0xoM_Bw<(~h&N9o4N7Wr=qtZ0vUW-N#2BrJ^yyXz=6WIC5f3SgL#kr3TH3 zMa+$c|5OU;tDLKyMlpKd)!D@LBA`pxup`bX+x#u^%KneI^i5Nc>G<^{!Mz}5QR+5j zddc4WXAvEw;pigsD@UEd*~K*$zAemFW`XZ6i@popsNW*Ih4dgOP;x9|A^M8(Q(uQiXpklG>R`RwWuUy4 zF4`m2DoycAevEY`@HgL!pKg^p7EabswB&=v%1xD zC>CAZVl#EAI#(*yOPGFlBzRs^guA4sQXnp2GQl$+e2RxCA}_w&ON9}~rkmpPfkz}d zgadC+7OSVAEl>dbaeudeW$LO_9PT37-SS$lNmJx2WPg+c-m@v^QqHHec|!g&fAl4l zf+$@eYp1DNhhRi;QsKX`oqI=n=6yo%7#=Y<)mvlYd`b_Bk##P27=T8SSf!(3!C?-{%D&m_MwJNnPa z!<{g9-+v-@cbZPtIpDv63TMi~4)*E=k&rY;K8)6|nDDf%2VJ?94%&1V>l z1Lggeo9jL>y&#;M?h5LwC}+w`M3H zR0|l?kpoG}^{aNc`s5TPLQ;$QJo??#{XvY{Qea5_?xhwStz|HD-VbbDic)M5m>X$~ z-w45@i);wLRebwmv_O66@1O5FdH@$l6q6pw%JnSWqPo|iea6Iqgjf&&X`H7!Xbl`L z**(Vl4_I_nsGgWJrs?o=&2V`inP%dDD>mNdC9CArWI3{lPcAitIVi6gRHcG@h$Zl*~ zY(oG>0q_gl4J<*5V;5%#Qw1$Zf@g^j#l_WMDTlA2BL|0gR07W%SE%Vb)e>U_RJblYQ(4>xOKacLCvwjUjkZfH{ zl>py)AAkRR8-;n;B;wr5RSKT&poqu?@nrC3x8a)Q!U8LVk=V;X>#^#K2+|ISG(Q1G zc6x_Ej^_7Mwf(^)^c|YA2B0~BSnvM{P~;6u3qw(7;m3k^bi?u*U8i7Q0hMH#zm(J zp;H_I?(e9V)GMmWS?#QJ^3Gds5h;#kGAqV0S5J|Bsk0BFdOM4M9|~Tw3e0E)HF7WZ zQM)foEZXv>Ms+V9+oZLE3oVsWi}?kal4we^IQ#S7MIr`pdiwx5A)RkSHlAxK-ZcP*#Sw|5>OJuqk2&+nmAv zE6>dwpo3Q;E>u7a?brY9+sp2^7NRRV#;DWeI?9y}(S{kKOylOXTrN^i%HKs_r1$(I z8C3a(qvQIC)4LHkCadsuE=**ycRnO*1MTHo`h1673JSQ-a{4@ez-!lheJ-rVF=lqGtk-2h#3fGF7m37|F9-(>eitztzStP!tevLWc zJ&}2w&}?dRM{~e?fsl3930dJX5fP)wCUi>Qof9~r*r<~|X&YR#n8QRIKZ4QaOa+s8 zG4uF=``irCrQ&(9HNS}&&=z>kE^0w!3aVw4aH=GqVK~*W;~V-tSi*?XS}DHNkJ|SdY4g_h@a@QcB$NNa8s zhD}4(vt$5pF{Cb`ahyZhSDWqEdL#~OY{$Xy>D(AA8^uQ*$Y(~-_A{{5sAZ(JYW*SeU_Mj9I@9uE~@$MHZp6vvTcvlQ* zEjFXGimgsw6<&xXzZU}E(tkQBv-&M;UwQTvT21zN2WX;>KJFZ> zIquYs*bX@kJ7Ux@<++iwI@9F)RsOkyRT*f6VoE@`Jp9p#g$~rzltRWY!>;MBMp?$7 zrBH`_LRi3{=3MY@&3Hu5_RR5Y%h;E7(Qm+wGWHVKM4eoJ16LWx?y(pX->4WRWmzWS zm2!>{Dl80nWA{H3<|DY>&KE%n7B7i0RVABHgFXtyj{vxvw$H6Ri*3zzai;1uCK>oj zE8p9W?BHSnJM&F*vaeQCvm5Vcqo-awacI&iE>S=V_DJCpOkr#eomFWT~nG$(?&=???;k_u;% zbZ-+YO44wy$ZXG^nq{vp7ZJ`XquKFk8i`ttg3QaNDMw_iuO(6MzH3A6Zp;XJ0*tt4 z&9sNR=lJ8=1mzE*a9?87Pnf@F#I|MIme|m^|HFj2=dk+lh@D5aEyaE~5u3Ejy)&st zEmv2hX?|=dcATqxM++sM*wTdkSlpmaE_@A!1P=S))I4m}8{XfZg>BDJI{E({t?Wq< zg}Zx{5-J2u<5GB#YbH6WhFDRpko_vCD%vyoY=HfU-Q(axj{};N)r{B)-k0>XgP=HR z#rC5h^NRlweL6X&R(Nc~@K^SH)`A=N-KYr61ji!xX^;btv@16s_}&zquMRn_F;Vuj zmx|_1->u#RX+s!OvS&+$N&!=}jkY3y>dyNykgZgdovz2Q*43G+VWkjRu7$wwvg_@i zbTPnVVhExl04EW|Y9Tos5^cEAO_k1bojcz_3CM>?j%LtPV>7O-ScjzUBcdPQTp`{@ zExTE1$BfhSjpJh+f3+qu8NnoZ;G61u46D;0W;G_wNQ#~9m4Zzw7Z}X=&DaC~$aP9W zo_S$QD$6`int-9pc#TWOxX_Y<*N!jQ*NR4By&KusGM)svefFPxa@TIj zxZ~=CfEW@Sq^cPSr2opy33%er_brr7{i6BI-USp|JGu;=qjTxd1vP_3<~zC7@n=ja zCor95BBH{*9S=8K&c<@}j9|+?rt0iMwOsowdbtrUm+CVl*z$$I-}a`ak*W-%8daOS zq+A~2FC%It5ui6dN16-NF9z=DftVA+M ze%hLSP-Sw2Hw5%`Lp znN{(11f#DE!w{daU=u_%Tce}PO1o%u_emk}y*}MS;Ti--q*0ApY0%Y zK5xUimeG)T)sA2E*j9(y74_6WjpSFi#yA*0&u$=;LyaWq_>>2}4UUZt%|6k^cH|l` zp8nIT701#rX8ust{59OJuDuv-AkSzzH#a3XA61lqtSn{cLS6X58whI8(eqXxA|&1S zNJA9%Dz&uu7M7#|FwuvAO!CD z;M4eN$xrR{EJmnRg}HK>Z=tkfE9luv5baQuwZJk?iP56T+}wQN=YbBrBfYExx@=)b z_JWmvj{${dDHy#SRB)3jy_9TLc31E0zfnP?k>cmBTfHf)=PJZ!3RC>g;oW8S1E8$l=vp@Wf8@*^8igjR zDJXJk?VJI&UccN`5WH(kcgW@TsveXe`bD%$MbWa@dZ-t;aBTz;-HZ&EC(Uo@r!)z$ ze(m%~TQDF0r?&r>wf~N8BfIlN@lz;-s*6CNmvq#c-Hp( zG~WH$^E{rNeiJ<7d9%A00CR*Wk)kq4Nfbp<5|yc3Bo#+V!q~4k1KicQ02o$RB{oZf*gv|khO`eYITWyg_JObmVvrbKv2%UhakDhz}Do`9C zy3Khgb*He@uuJe8b_+|*Qf@)mvrxJ!)6qEK7->j+(dTDW#XLIA`XAo>dD!PWu7wlW*VT}e^N$rKTG9I`~TsE%2JLB@ko6YyZ{bl8(?N790) z6H$_?VN_^e_-%z1 z{x<0L9y-JlzkX7*-(-sGd~gMRkvU9KI%$oZnB|zmB^XZXOf`cCZX@5^-exDd=2Fvh zeF8K+>5fGP61pAp%=9v&a;T`mzTg+QHb9gil1qh57o2uUO62aFxk0m%Z<;vlxbMY` z2w_kR;Y*&y-q)?t$x$0bV5jSH6;pd|2e)!hJE8I&u7oA6JV`;CU0$C_C`f0hDN`u4 zrL52*;3q6Wu{r6M8~c!G)M6<+NI zp1hB%VXB}XtxRlr>bhkIc*rMR$g20 z&ho*# zxrUp=+JXp5bM_n*f5A}7+*A`?eYejol^>xHy;^7%)(8h}ClloosA@nUWVTe8H^r$; z-@ISoq5$zUqiJX@*~GNwOBee?JPCMo+de5- z3u0eg*_r+3#T@ZuGzFanDLvIS z@m-R=ZEXUj>v%OIKy1xB;8mk+%0=y9xuuENUW3s%GXg zeTJ3RMr)I0m35eV7+S?k^wpQ)-DHK&{IA(J>=29fZoQ;RRU^)Vgd>0!XYS2<4WU?`tfgw^FIR#b*6oW{Ra6rFK5<1oDCz#C_i4` zJnxuZy2olJ>hGDqwi(EhtD`*qd5otY`aECA$-&eOlpg#NDyM&r_Wj0ZFWQXK_ZKm8 zH8fd8W@NfrrRRiy9qk6P<{j(>V!)JWm%7$0UJN;4hWz_wmm0asLCLa36=EsV_d<-d}y{CU)|g-Q3QY$dGIPlaS;ypa+DsfhK| z%mSQKNKc`~646sQ^;dYgNVGv+l>ZuEgc;vrsgek=!VEMnJa-EIZ<(>upkl#Ou!iXg z6edA$alArVox5s1`4Z}mT;W5Mb;%H5!ui?=Y}EiGdIQA4myYa_VJftc2by`Y6vMGQ z7NG~RP)JmZdb|SjWDW}4gjl+rWVAlp`v;=4IDJIO41cIsS0K9W&@3B?f=2t{=Abwd zx&|qY2Qxkpnej8hcM?VuHfcg?xb@7j8&Bd1mO-vu!JRfamGu^Kqi9XjvL_0=;z)v} zhBQ-*Ij7bP)tCu{4JG}5@%Mpx(B;sNOqD77mG>f0Ww?kBN~1{y^m{bJy%-b55V=t_ z?oCxDTrvytzey-%o_PW{ToDtD}yT*{+5(JU5`P^t?0?XHq6|5Q9Hy z2E<-2EYTB3vJ>(15?{r`6<_SujQ zjR3b4$5i;-1UPphO4`ot;0E~z{8D}yFK;B$?5mk5Em+Tv0B==N1QTcn0tt-k_xu?# ziI|DfZj8)Ur>*oy=8e`1MMG5<7_>y!O5m3MlfO&~7osjaqkE`-q~9m!baI8}qRR)3 zkXfWDgYgBn_T_3lq4|)7Vy~nSR{h2cAHWj?M!-GO{WQ3>A6uH zAm|2~CXTP%KIE_54o9dtSmcb<1)xQ(vrufMu}Vp1D;=}!b-e-LFR3|fnZ|9bU7&|K3UE`H9d3F4@*KWe7E*0^=_^baGv5~vf%-}jqARDKn zvCs6;Pt2-87B2%od!07nXYaWEw@XtZ7ykY&eUvsEap^1oy`&-gE!uE`Ml?-BV{eRL zXHNqhGgYx0-0q)sFN4N``K|mFy<^;q14O``iN;^#Kl%P`LsaF@LY#6F`=KQN0A(NN zM)!@hBlU+Ud+8|uRH4DHezw_mb{`?jNZPO2RN{Cp;S2{OOIpb$;`%-nO9R9#bsVxp z3>G%oH`@ii4o~wH`YqD2mWvW@86!&Va^B0?e5Ui7&0Op1n) z@G&RPcj$?$UC4UMx(*S8j>VgFgYbGji)J9&!c?BU%E*d)#~y=X?nk$d#vs0rmm=wQ zC$52DYZ8)%ZD;t7#rnQd36J0#dC=6W!-Xzr@TMYkJEbmZK~e|X$p)95tqn%v_R(tY zzD_84w$B8CekvdjRja?d_n78YPya~H(X_0LC{^}x$Z{f?M8D7)`)MfVxs`(b6o`g^ zCjd~PB(635^j2)r2P#AZiO-8_q(!g)Y!v%g1=7i1I>&f=CEXQl5I#mz$wciQ^A&TS zdBi+sqDYETw!`U2h_LMMq)1E@)8^myNQcm8YE^VaM$^74MlvT6p6L}PtxD?9SY!!r zkqCE~3iR<|go?>MTa?@_Ff+`YR>i^dPmeI39!xxzEEIb;MMV*V_Hud)F3I!un3H1F zf4~W-F3r1oLyeAp!gFgn6gpqCM(qS(r`fmRL?`HhYW`eM+{mzO^Qif7K$a@g&=5MOerq{IJgi@ zvf4+VWKr1v;qHn|5DWmVl!;1~>V%JL(7oDpZ5&aFWgv%i9ekMH|6mD2oH4oDDu*q` z=B^f_T#0NnnysCg{e?{y2#J~{@>hRoZ}t<)a~VhokKB%NGDyE%r#x3v=#jg}c7MIu zNL2V^B9QHl#Lh&)v^=p-N|#!hU}(CR@|=CazGTlcuh`Kj*3j#FJc>D-3=_@J7=bc? z(QZg6gnhIT-rQyDp)aBT6(4-4D|O5DN-9O_tDWl^87EwsXiQjhKm^9!pAR?RE<3cE zZ;#Q{Wvh%Zdr+RVa;ceCop;eJs9vW=_`Jg%I9i#;@b%UZNh#d2$GYJ^(ENOU_hMa7 zfhgP8W5VB1$El0o=++J7!$>6T<2FLaVUF$K>)RX@p{n?41rjrJn9fN%L{fGg87k_Y ze9<6-s{ojyp?Z~a`uG7!TU`u|A|jq@^ji4qeS4o7h{kNTmK2WHR6Q6Vu(7$CpUJIe z=D%2_BL;5635?{C4Y7t&3(Y`|e2xbBsbn#>3Zwk)Ds?;8-XJxJM~^RgsNY*&WhD9y zQK+vgQ*WNy9t$m|cTgH_^{19I(@oR~C|)KV29v|004o;3`;ACl(3}WDuy|rgK|j$a zQXl9ZJ-rqNx)n?)HA_@cu^4F9;bv@tDlikp$tS9j9?iu0tu9DBzln0&Y!Tqh$ZA)0 z*o8|Zak)IrK8q=@yR(fYV33odbd;6z{HY^JF4C=cO(QPqFwPF!*sIK~DbJ2+Xq+0H zFQV8S`inz)cjYX2f#Z>Fi(#u_8*nta_HT3^fkRNUGqV02yc8M+#^I;h%_xLd+mfoA zpgSBNjMegjp>!D7MzB1X%N%Ygm6}K<X( zn#xS&rpLx7#;0ltOVO{FtnG0^t~>*|40*Y8B_>}={d2lBt~M7uQh#l97gpXWxUznCUJI4SCg zCmZE+ZAW)L2jqK7@F1kM=2S+Q3WANAGs zg)L{9E7$xy{KPaRa>_K(d+{ichNcB`I3+sUWV=@hih80o7O>&jC{7j@!)9rd6r%IcoR(>yl?Dpu;gv^_o9b4EiC7PSTPWUzF0Z!7p#q zw+rhM&W1%;A}!Hn_nC! zqeW42npg*XGt}cxcje_2yEmWdh z`t!^At)$Sc8Up&o_gc#UO3nroVJ$G&Z}Mwg(BO%pKeG?;F6b^spBD1> z%goPL`(vO}GpsN-UnMd27%F=F#5T%A!ZH1EJ$+jD|2?h&!!b6R?`Vs`vuHMS_C~2v zNr7~;H1c!PlQducY>3N=k#R8e<4@YlS!hCz2B4pY zY{9p8P}il47=gX6Gh8~e=35XI)WAsZ^?uz1V5F4bHKezDD9p}$c4sGvsQnhof}+jO zT{dZAr7iu26Gf*@VLKwJ@K7xhhHi z;1Gp(~gdNnN_eg2Bs(D%uW!o{M+ zT~&E0RCH*pW&$uG`9c_jxSi%*=I}{!>^ch6i&kitrz882h3?sG&B9giZbHxUg-)>) zYI_WLt=|m&xD;gEhr&;=tc}6zpzWgJxQ{b8u`t2VvAOwzjYF~bdG1&`9?*3Idf4SCKCD6u)IDPJrlhU+kI}#l7c_#n&c9FDW9M314Rs%FaqVxW5qe=G@z50B8fxb{*qz_9#L?h$* z@T78Y$o(!Hy{1NIAn*m4@l{FW{D3Z!(6t-X5;_Xt9D)p2ntQF(!9!PjqmRA{68Y!o z<1o;shrP=b8j1|X1|RpfoqROMy& zC54kf$l6Fn4U}tOdfOYQz!Y|?B(HK~MwB+#tl(1$cfV^G1-_G7>pNo4Y#AVoaKhaH z3R@pqP=ODN^V_orLq541y4jH#G`3w+!AgC z^sQTD0uqR0R`5rc#*oN5g&^rT%Dzm+YeY-gQa<<3(E^<>(LXzXj1)qMO0X`S-O=qg zKzNxLO>z6WesSFZ5n0C}i-Z8IKvTcM3@tDk!Kx*QS0ge7bK4Pots1sOINneJ$kcBI zg90GmX#`Ft65M7b#^9p!*{kcHnxM>x<1gu~eE}iJ<`7#f@ZaxxF{*qDOiLAr0s@V~45J zw2RrzEU_-NF0&3V`bY1&n?ab>G)njVo}A__JvqernTpVg#0E#?`@62&eHD@}&1do+ zv7D+n<-v>R95EA_pzZ%c+aN~y)ZCZL+F(1_qs<)?E_m+34NQx6Dd$~r4EovG(MO+& zOCMii2yxC-g!Zx`+s=lN-`qOGdeNf?3=1fk4P;f85LSKJvF+TMHg|UG;uEyO#yEJ@{TcxJCGbnAFD*heOOXh4qK-@ zO}S!xmJ-}mG%Z@uO`%+y-ikk8~sgUq$3t!g!fT~mz_MNmhnbc9G&KH3D%{Gg>1 zKKHX|m!eB^!HhR7k*IYkfBMVyFpF?SyVS62c^PcLy;oj z5g#myH6Nf01q+c-Wue%nI28n>N7VV< zeuhl^<-=EnXPIQwre;u4fElq|Bv{5-D6W#JVie5^liMXq6%S^Gsc4MX7|n^gr+2!) z62}q}&G~O53IGFLlx#DuU)%vlUV`+Jhox!hmrSD|vQb=@pn`x0<*mT zPMd8-qzWlaHg{LI=u~Z*bgvfSTPSzAVTEC(A;3f^E>Sv2=mhR#>Z-&X>A1*1j)7!_ zA_aYdXCh&XeH%w3TxmO7#c55m#?z6#J8@X+R%!zjVRX|I0G;O)Wa>8jGE6(Br8Co& zgONX5_6{)}(`#b?7sb!~Kdv0K>j$lIRDHkQQv5_utm|}IFy}${+e8nym9>@EIMzjhEe9r<_ z^hmRy5RE({ARiiLp~7Qd5anI)w6~qHowaqDVRO=y#$1&6Os}rYtuUw#)5x~6+gPQe zMlsgoVoYc-XR1P0_)Wcr)(Daeqg(?zg=> zKHos74y7Vg77G@MbS|oDL(i?y4Gy&va@u2&rAS7jpenk|h*GOOsFY2rY=u@{NSaYo zL9wq{*8?M_kP^;U77%-%BXW2S@EI9l`Y5VrW89m*-w|#892Eb(Prmiyqpekb6?h7 z86s{K<%y??GO+-nB7Wt-u z08AJ6xhP>DtIlpI)GysI2N8Xb$;er0RWUIEEg)HEIvv>uR%arC^79VG z&i>}{EgnCFekbBq#Wh&eF?{2Zb%>F@+E?tsJI{>7okvw=rm>e2_XV1GQ2YIiS^ZOIp=HtU70K_l(kzldo?vpJ}Q5E$p7RF!wp$ zZa#bP*ze$PM`I6mTYSG|=fT%3{8bpC36M0hhw7Py^D%J<%^X$p=~+yE?Q;pL{e?Cu zr7)=|0KF}{kb?M90krjHw3_kAcGlEq>eWMPP>p}+_h6pn<`2&XBLG{qxxDEc9nT0K z<&ZrDi8iR_=5l@JVe41^<`KpZsG?wfmiRL9xsAx=AycNh1n-Al;k(4_8XrQH{HJ-iF>=?`!jb!s!qvDwCu70j3D402# ziWIbrq$;0|X=y(!QANeWq1N|9GjuDy<^ca9*Nz#7EZv5qx?{TI!#K)(EOcSNj-2j} zGG1^Ezq+xj(Lp9=Dz|Dlw&%P-x>a9@p?!U1^(_TlJWXkitC*^!BgYD*{RV|w%teV; zZ5lN#{j_o;p14X&1AgG1zO6{gU%!ZYEv*+5k>6K1P-UaoOU#$dQRWy!O2{fTw8tT{ zo19Mgcd*430m9~8E&p{azTAWec}mq$Zbe@0qkba*w9=6SJ1@n28FM&(sr5*_lvBvg zt1KjtSyJRwt6>`UuIABeLe9vHPPV&rR-GehGLLX5x9B`OX68y7+k(x+dRKhic4>?rw{+4^)l?~ z!j#~)o4FabU5K81ByMT!UWdZ@ z_V@tXRpKWEb!H$}89UN|%Y$`Y&{O+NBXgp6$6x$sfK`-A3mOE{SHj_zs)LMtC`%=$ zrKV?(RvBQiOhn1GdaYiW$!*6avtbUbLE0@$DZc3^(w;=_swEhjuHV+-O;A*Z=oln8 zCX^L@w|m!JhG;~G(R~1JBnq6efI%z+F!IR6@~Usl9c(AaX&FcuL=RB6`4%GVVvMHe zZ~jE&Pie?@KqqGkwC!W7_bnd-Bcw#G(8(1!1%7R>aTj=VC5>kDQLl3<8neZ+)w0cU z)x1jIq+hKMn<|h}ukwl`T{-yRZ&xs#cLPK}D+0Nrn}O$;Q%i2?2ztN-O$jva*#Gd) z1&=P4A_o@Da&HicL2iiJC*cgamd`Pu??S#CN~=JHSywu;1Bf7U0&LK4d$>XllThFe zA*9l<`6TYPom+%d#D23;Tz6clMKMq33$VG>9ncZju!L|zU2ToH4M`?>lt2XzvtXh3Js}5@gxF{Cj@^_>?o?Q~npqF^&T5 zpw=0^Qp$La0n*4gT4E3dX%9=Ok~}%-E-1?Be-5B;nfs0Y`P{&tNxNhsN(!SeBBm9@ zj_23*GJ_D!L%#EprMy(lbxt`}d(#S~MW}_kO4t^=5`|l+t-ZE^W4C8Z(zGsAMd+%z5QaIp*G^#>?(DKi?q zBBi*QmT%KKW~iW$^gByK_K51*(YXHPFa&~pG2vuGCjsT0iN+T@M|=RZUSZgWw!*wAPDPT6F9_(w0&w68!5yOw&FI}7 zHOd&56cZ{IW1_mCntrh7>O)q>*N>M=0887ki@U$&JR)Q%qCy96Gwb(uNbBME5&I8+ zvYP$JtsfpNsVAVU*%~sF3U*N{LY>?(6zL7p;)=MCi}< z9yv$`XAQ1)V4}ZNTOdC5y&AM8*ny%^+4ZRE)v9|sUs;|nXn!M9wJmfhI0&h75I!Rn%sns ztxExpqdBD+$g!KHUGWvAg7$##s{YopmuC4pKW5ut-e}%r-fV_tN2{ApjG~jRBGjba zeKz4)e(Cqj3RcV;VpKece0aPsU*ZtBG>V^u`&H|01oR_HCZCUW#RYys*-YIk@woZy zy-pjk^sLa#!0?jI$mUh;V>95{?9>UrC+L45=8`4qCW&L)x$qL;_zSyV3 zbNqU;9~mc?lZXU`UHb0rwd6ul(bn<0#JT{{>3SEXmpGrqJdJr46E*kTeu3%nh>%dL z5UJNaKsyFY{}R^RNE)?n4j0dY%fE;6y7(HU?Q*pHLKW`ky0(Sl_oYWT9l2U;5vcpi8xOkdpu&!5!y%<0QnJ9L(RVfH>x@8CeAKnnLme^xfITzn) zE-oMg{4_MSH157`wLK#2rEqcO^RXVokUoUhjR3_Wuk|3?n}3i5UuLN)11L>gvQb<( zLB4_p&{b3y=e2}unn1J3Uo3I0W(%L7=@;YI+X!Xq(aZq2zNFF>Ef&Dpj78?ij4yQF z&3fwfumov73&kF=shA3$T3{#!(DY{t%CEvmaQGd_VDws9XpTlNg2W#%B`h>8 zQsbNOrTAy3vEM*6%MuP~$o`#Yi>j{@P{gswya(+?QWt<07fD5^)CqWqt-Yq6;sC6r zNyC|Rj9n#@=JL53>?RndP*JGmnfhDvn;_5338wrug3{5oah_S8?i=JgXP_UP60X_x-7a-01oeeeDzJrVz0$WrDcUrw<*{_M`> z-;3NFD}02qBKS9H9}Z{DwM-uQT+K&)D<4M91bqgAoKY%}dA7wll&A0MzwoSP5b^8yYAv8Vs;y{a}!zR1la)xQ1 zvz858he|~L5&mW)TGjI=`Ih0-;{!J7Gl(Vpcdh@CEE-KCcmm_Iq~uO!8hBBzQHmkjA~Av@$PWM#ZnMaz(TW#3tI zR6;aTrzUvYAwVD@UDfHHQhQBT zR|g9CL{nihko|y_c2F+G23(2_ zBw~u%GFvj|>;^L@A=>^c(yd%~ge5Rt;h!MWZjd@vr9bnY_7fF&9!g!$PmV^?FTyc{ zZV9a$DzK}Zttg+KL82z88ADo&VmzSwX>6hfrG?fujnX@&Ii68`Gp(f!<7FbDNrwfE z#Sd^{-S{r=43 z19xojanY}?oK?BcZ30M4x|+I{X%VyG`D}>iX()zJG!HlSXj}Lk6fYyrLa&F`e#lS_ z&nCA8m(PYhqavBJS0E&FZXaQ9!r<*Q^|+J%=ydT|X;uEcFb$=yWTX0IDRhf89rbF3 zdll=<@Q@O@lLA+0Qmxr3(NwZeidH#!%uFhhYkQ5#*)S7N zBO%;@c+9#jiyXuJQ%X*H9I^uOAZ(Z|_oJDCTvyoI5t`j~*w$7f8Gn5deBco?Ngxr^ zM{N=^VCL7s-15+WAA}ZzUnkdxmDE=M>1;g!i+s$edanN{pq0vw+saH zEgQv=6*(CbtUYc&oOfdpx|%HIQD_b>XUg3HPHIpnj_&o-%N7Fl!4}N4e9l|eh&&VV zRX)sNA10U$>Bv6ENe{$YdUXN7ScJ~ba{I{HPQ*SxqTVwK`}~P0@s3Vo7oR-9__lV$ zKq{M!;$?1b#Fmp&?M7ct({51{P=c%jN<6yty~aVev>)=`hDC921;Tr=0P!fIbEKj$ z7CJxmSBjf-#8IDv5-uCaK{A|S6f9Xom-6=HhMgV6oNNkmKD9*QBGUQp~bMb+igF^A|AMC5og@Zmx;w{obFX_lG#b4<* z_USMC8wi@|dni*9%^N`hSw26TF|zUuA6Q`MS`mmeM+X_<{RpCaQ1>7@qTAHR6gAF* zsxS))6-H%IuAqmELzWkgeHN%6clTQSl_H=y-_@{nqHcZdBR0?~-MXmL8g@NBy#x_2 zOzovQSKc}-@j%3}!8Jnqcag|8(SXZD0|{SJ z0O7elUWu=xwVa|s8n`3Y{>~EV!1=MdP{L~N=$8RPPc$9b-=mdT6_bsLD5=ac*SRQh zapGW7MPf_*+IT?h*5Yq0&JW4BhU21Uzr>q|%S8fMoQ7=oC|}>Bx+80PVu;~P26D_* zjBz=S3m0SACn@!l(=XwDoNNcj^7m4-5&AO z;LqP?{KWyn>Hk}3;?acpY#lo~?_js>j>7&aED|z+fl3f*XS`W)rIgLc}5$vi!prgj0&DmU-|B=``c2W*5~05F806rlAsoix(oTM#c(cv*YhN%dc}pV{L+#P7Qzi2!jix_w8qd zZ=hTiB8KJ|vs*8EU9g-q&PiH;Z~{u0{Wpj8XLuzypT<|`MmY+NnFUuqCUUe%C}r6A zR24Dc#%!yU#=3;Ok*v?SPTivGusmmkIF<2tlfpUkm+^%rfh#lLZr^W!v^N!@c}`%& zda>RIu*r~2!YczI1Hp;E@%yCk>t}=Mlp;uHxT;ylO zS~z4__57jrUql}yEo=ne*n`vzLoeidwqk-Jn~G2k!Sz{ze~3#$#aj5*M({|kObt}{ ziRMtYy5%^(Y6s)#lPYh~AmpJj56of3TGhd~QfWjjtB!g6av_vTKNhAB+t2_@R;>#E zgLkeAiVAI83?s84R9i-{*U_%rvvmp~G!zwBZQSIB){qz%O-U|m#(uDyU+dV8sz5L;GVSDsn^LU_3$i-w4ZfD{FPedw#36petEhvdSJ7DUyn zW!I?C8+R(vB)I=!JUn+BnbXJu@JUbMU_?CY}7P1&O|D(MEJjq67Zc#0R8DEHc4m zx+oSKRT3cHy`Sx?mhh`RjPL=m7b?5FJ1LAWdBYYF(N;1nK00Zg! ztq4-w`d`)-`3R>$^;ed*^zdc?!@d(-)tk9TCJNxFdJIxlmBd1&m@^U0sSqtesX2AT zQrggHgrhGT#on;RDWy%<)iC~o072DKs3Ujiw>6cKV7Rw%lT-ldLKbe21J+kE{xUdW zb&=!^(mR^kAZbQ=u-yj*7YB-#XF2(rKbGoUc#VTsmW^TwRURk`26i${rP8T7$f)~n z{KV1wZ8T{M`-)LTuc|K7tIZQ0djcSqs>t#0a9>^nWv>}RGBWN>3b#q-;Xm*4 z_>r_arj6*I#Ldm_*Z`?Q2)Qh&4M8X>;rF)h)5B^Rk8HJ0`5dAG|KTo{W`10-uD(Ig z@Tqc+ruOoMd!a^(7d}Q)JZ2J4I^sttbOyeKatqnfJN6#HW5lyK9a8Dh$xhxf;4bo(~x-Q2KV&*cvvWN0-QJ}K+Ig; zLIvjTHQTYP>;4F}NEV8l?^Me}-B`auDi=ciuG&O3w5tajdf-%WMbZRyRH-qgX5V+@ zI4ik|IIVg1Ir?rG7#2n}YSq(p>#GD@MjQ>5#O58H7u=E%Vg(D@^K(UQiji zbYr5}40&iwYfPc7$R@|!01;IIa8!2~Q?N6WxUU@{utbr_;v$ii0dc~txAnSKodoRZ zLtWhg;_#S+lF3SvTGTYRCR0t*MLFU42xT|L$oGsU`Q)Ly%j92*|0JD7+2OYA7M`B% z(Brcl5T>ARy9}Gf0uii)CesrTdHAD!ET=>_cHv(Bo4-+dRiZw)9p8KZgJ`ZId}}Z3 z%MSnvCL6`BO;FxB3T9#0jib!#X%Ke?ax|D}r&1Mv)R5EDA(O=9__?G=yu3b>7!`yR z5`RN~peb#C^-Jj21^0%1la(Glx5NhuXTx(zLEIZAa^DnIMnzg{Uu;BLizbz4BB9@| zY@jKRqfi6wO9LTdA^iXvvzvzObxvv~$YIR*q=4*-(p%u>q@DNgBSLZ(irderfz+rO zyILP6@%6IR213N!g=_?rEYv9I)!{;E>07`UL4oF?g*>OAlqcq=W&lZCW3Dqd7|{G< ziW=NJq_i_f_ESW6w|}tdba87>YnN3^&sza`*kDGlE-gcbksz|sU&24Z4VXGv;tWU zid*1pMasVP$~CBUb+V=pmZGEx49FTgr~p-9Z(-a#&UIu*>GZ=h`T$fEpPb8X=+`{H@^Wiru$U^;q*S4&2HKj;B< z?+~3OUS=yLMFe9?M~{nmxvK5wnf-ye(itTFBDe__bHPmzEg;dXi}A>IPLO*WJv_pT zMHo~&b5O#7{i=P9X^lylQ^}gJQF1E3t*{Fd01Xv>{IzCNSGAvmQX>;N7C99fE_ceA zKt+HsT1Y}hpAgo2rxMW|6esKNOEJh7(1{!ToNRh{G@hEvDwL?kAxi|2_0s;zF7QRz z*q0w*3Ed9+V&^8VN?#$fE0l%LDN*YczO9DwoCe8#qmYNj-WWkDjgs*Wjit!|5A93Z z)^G|Mqn4f#09wvnJsYmiMseq3FT^VS3cMm4a0eiIW+3}lCQ5FcFT^Y^#6%gJhU`n6 zPx)v3bN&VYl7Gd&<^|Il11_IMro#z+DPeV^24*?)rTMZsVD2{uuUu^NNjkJ)MHJ9| zpxZ2aOhx$>R$|IN7NIIb9f|04qCc?{ zsi35?kU-85BgQQgUR?l|N@!}Cw9MHQzdF7-zBztPyd2b5hi79i zwn$&D;TYGgKYd{vss9n(&WEWMC0GW2Cs;IyhWUCWb)Gw5J6i=@rCcEk#cnoAqBDSB zosa+exhG&IIX=ZP%1H$EB)@@jRl35O!GM|}s`a_*XXN)lA{0Up9??m*j0#OcO}_{l z*qEtm<8VP&bPdR{VPNZH#qyyOWlgLPw_n%n-LI2PgH;Yk+p-e_Hc5cCAk@fLHQO}) z%h_dr19+wkHO!M@NchgE)c&|TlMW89vdV^_ekEa%PSA5u+y=u&gOZc@Vk?VlFdj*D~w{RZz=uUM3@B1^sg)H^X&J-AeUuo8h zb45Z}Vg!){?<`y*5_G~bh$|N&IYRZBH1}fnOmcCmnx-pzu9~6d=@G`$1JoCTFdoH_ zXtWUM?MMCiHv~*09XVF&$hLsGFGtK_+7FZ(lVzdUTD{5|?`1E)$!Fn}Wg-EVIDpHG zj4#ezW#QSYHK3*xy}Bjg0>yN}g{-HnJ`2HNrb&X)$V=RjbkACm7?VnazE;yEE1B?> zBrWV~ClC}-0Xn~3-F_Q509-IK-cgykQAU$dEbh1KmK`W!WioitoTrSL{x+#=^+j-p zoU(5tS0!JB$3Dja;Ag5(PiUg_RAQ3TkJOMY6y7STVA zii@bx(rr{tO2+OTY{3b4EKGWGQ9=alN}q*_Qz>dT`qHrE(qj`5P~HM6(#j1TYUPM; z(&P#(vOm6!7cvA&ZrFKVa&HF}fZQgxBy|+Z-lRS(R1Zh^?X3rh)iQaQtrlYsE^sB$ zv7K2Km<@;KR5T_$0BmqBcdl?g;i=4@8$zwm>Is*MEF`RSUU3XKt~yBg*q+5qYk7c} z_-CoYb@|K9jJQ%_DF;|#8XCKjrOIRKmDl0~T!zh39TN^QVZ_!jv->z2ZrT+wOP35t zoN}82=6sB8+jKgf$lRl+o4Urzf0r(vDkO2Q;<+X90$(cX@H|YY-+zd*qARO_DaEqq ztmjQ222!*cXx#M?$hLe_HG_+!T0JC=qv9|_iE+^i%h4p>#tnWs1Y5-tjC;pfjdXyo z#A8ISG_`lB)Aknr$kT@9hN@sVrKuh2QVOjbf4)8-oiMY7ypd!xxg8S@?SGXBd*U4p z{v7pZ{n@Q=uA5=0=Z1ar!k~WIHl7mMO5$31&TUYsrxj8wTXh42?Tjozps7a+j@1+e zNKrl^rD+_KQA%ZuwrGb>wEK(w9MKtYgwKT#$0Cx0)ZyCYk|`|13Au^6s(of+=`}pnJt{A4(Ik>roB@#Ru=Wbhba4%m4GQmdD99w zmO9sg^XjVVL5X*E@KyS)_0|UKa%;HpP>K;Qz%ciofD$5bV)}GBhspxbYtsO)rYx&| zl_YJ#e-m5?t#L>yJ}Bko&3v7$vY>{MrpIcaer}zqw|gDQBV->(y#h+7>KSye|@=tPjG{6f-l ziqNao6atSkO{a%Q|JY3cMelIb^<3s#UKim3=~jD@m@7Qh&G zWK(T0RC3j;%1yAMhnvhb7Sfnos}-l^aqz_0(R|uTu%BGCWMrwKx3JVeyb%Rh&uzJB zkAU@PtuW1kOYzPUxJD=UKLvh6wdr}OME>|<#7ML?3s;E*?@Fth+MK-jYClnms8>^_ zZwGb{8Hi(PayVbuKXOs2S5#+>ngPLsh?(QSlVTGq=$&f@h~I`fONHerj`(jjI=&hVj5B zk))Zp66$~7cWR0oILKGVBim(bz`EWFw|Sx2u^4zYNu$B8^d|eNmZnqLbhn;sDLV_YCa&%Tt!xx ziN=!+Lk(B+R@>gr20f%oGmwkSBhLyO?U(Fd+T|=ky5O0J5_db6#HxaVs|H!T3~+!t zVJro$rN?5!mL!oBtrX|@-5B~`w9Pwzy!nU3sh6ezadJ)iyL;!Gh;%<4*=4S%;gCY1 zFEj>~rwnx`uf@vNaeS>_wKTZ=>Ao-F6ZzpqSz+E3rz(g=In*c`21R+^!vR7ic48El z5pMT??J1Eu2LatEd?&&R%F>h64O2fgQ+$47FEi+qZpg0_-b3k^nUIM~bX|-c`;5=m z`XgO{5MMPmq~Jv*NBpi{v#&xDQPVKFy82T;j6{1F;rnR%Wj4TS;FC1LwO`?jjJOED zM}m-px$;VDqqWJh$~wG7Lkr6LKCTSNhpQ@KvDmbvvHg8wX=bWPRq}It81EBFt_d53 zU!Wh5F*(e~DK(TC$_*8o|9!?(8dNwra-z$3(KK=7}5unvmyG zc-UOT2+7E}+S%+3Cb@L~Ck2_h4ZjRe+UaGLjEvVE;e4D9nZq=PNXdT!$qj$1aEw?a zng~`zRGNbdWJt?|o`wvmHj_SS@-CC4i}8DF(Fk(7 z)!qM(baCEWP_z_E{S!2`6-<_Ahsxm6iNUn7!=Oe~SoZBz4{pRj>XM_W;cV1xpChez zQ4pr1aT>`R<+ZNcWbF3doFrXm(ug7# z4VzwHnM_3Ku@Sk}1(xC3NghO9$W|jKlhP{q6H(#~n=DU-TKQZvg9|YsDwTzVc@||g zpRZd|=_6gBCPfo$D`WbfHb@Q4notf^PT)KWL_@$6&=FOl5J5}cNvYA*>N-#-QX18$ zmP_0Tx;H)76FBrV`A}feceK~~2{)@BqM4W3 zev{neXEaQYRht<10YG&cG=5HjapA6}JYtq^%o`@dv<&NMY>FQ3$4`?Y|Smu-27~~k*Y*$ zaXXoXx4U!%hVvsdLH-@lzQ_1+FaDT_tGRSD`w2V8Fs!c3xiRCDx_1W z3rdrkNhrCX^m|i zgpBQnqxL7~2YZEjuR9$Q=+UspTFd_CKy$%Xg7~fGgz5mcR-$Ho##yC)7vM2|h~6Ech>4;>Cd0JD3*6<4ali17g+e?B z#mhS3u-bk2JH;Mplwl8bUM+OHaGk+Fx1Z=|OpG!o=bp0o7zC0}g&D}T&N-6ixffen zLFC&WuI=Ug?XIJ--Y4#9&})ASp@xeWP)~b+ka*5SvB9H%O+rh8tVeRDnQ4_CXOzR?MVi4x^L0VV9Rstc!_ zFzw&$Cs^mhErWe}H4oXVIr&F{$ZnQT1@cHfUoacGQ0d5V zKo_CIZ{~+ns<}KangxBu8OSwwE5uJes(hCc07G{$R;=HqYy|9ewI zU^0QoQv_=xH{btn(prj9*ZSG{OL2}!Sl~}YN$Z`j5?h^vNlFD?dY(2s^ypRN2Pa=Y zQ40eSyYMYE@mSJJ_B`{79hKoV4LMZ$3BOZcxJgfFOscf@iVd&V*2NG5oh)^epX>T_ z7g~+Pyfh2N7FsM@+2<6bsZ2Bn3IfpC)WEOZu3$RvLQ^v&OS!?Re4q>P_pas#EQG{M z7!pkl{FP186lC0rG?r-=iY4T3p|+r%Dn+g0DbQt8yv#xx9mNx{UGVOQY3R7=^|(DLl$T(Q-Mn_MtQ)`>~d_ zOpC#@Xf_E_aKL0&O2Hq!>u!cAmPtET?>B6Jvo?pQjMGreUlB7#b^JaxT(KuweWg)cS`Fv6D2mJf2V2}WKhN=l`I`4ma34RW8^d+m%~COD~rlq_=;J$qr3L5U(} zP0C5$eEh3Hk7y?OlYPWti6KZl2nBL;R_IAE;7}$ce0S-Dm0M$H!x?0-h2wBhl-kWpMG@{q?~!d zg&V#E?v>k`;9d@E3aLmbJ1U33_vmxA-*VM@@?{6X8XHHM3vlO{U3G2-_N@z9f{#qP zQ~pc^b7Iq;Dew_#N?~Y4k5Db6(J#LTe%#_eMQ%lUZd@C`Y=5aqhE5FL3jCO&j}GnK zC;i@Jwcp!qIl3iYEdeF}RaibG9cNx{Rr^ zQ8u^P`Yhlzz`Iqvx1HZB+S)P$erRZc0Y70YxA8_lVScajGnah0u>PQp9Hz^yfOo{6 z*)m{&4kYyr2aKil%cCvfUAvj)Wnca^T%mXun`S=j8jOPnt+pyY*tTU6fE=C52eEp> z?E|OdWcLyL7Q&rtE>pDWVFvtA#hor(%?w^zG97-Za?b}|Fdh9cmog|{d5Qhttt)!+ zGvu29ZMvzycDH;X12>|!!cF_dW|#^6O7)t!YtQxoJM>4Y*NpMpD+ZG7hpN|1UpBNy zkgl1cddXQ zRbBKY+|mF0$eDfufJ>{Oatl9y+H>fAh`Wm0SoDvtiXV@M`&7}=H!)A|)G?Cw?R-Xc z2eCJIK7vX1pH#2WcRj3uxH?{2-`W$kX&z%Q+tI3)At zrT+QqPqtmFNrjJBJc#bo%)J8_{tT{A2^VkPeSCVgojRQ0J63+DdB?hJIDggFp}Fei z|0Eb^YRkv5s^W&35H3->CUMOg41xChs@II~zIsbikH+t-UNgbl*8&RDlT@$y39j#m zCn&ON*ZlX^X93`HjZwYkZ@>0_4G8W8jcs8VDodBu5=uZ`2~%_ErkPOWQ@iL8)826& z>@f_=GyL+Dr-6V|mM+Pc_0GA>K=Z-Bgew%eMDhQ=(X@Q3VpT!L)t#>^yteiyuw=#{ zRA&KJD8*fhqUis}efj%IzJ!KCFWzo!^7#ou$q4Ym{*QbW08(OwX(Kb>Ol2Rnd1jO7 zi1?5WsB$K{j2Wt{A`Qn=5eg5AIubh*1%5-X(Bu|bLSs`+fRQUg=WbqM;VCCYTv8RN z=UHfSM4V`pT@fFiRPObIGylwyu@tE5!CyUnwy%j8<4i^=UX%7HY8_vC8tS1WVfx4q znOipD@*=QUj2EUOSG#VLu7Zu$5xxIzkMw2&jF2}>=!tYbR6Q-09+?FzJmN*Un$z^6 zH0{%sZJ!M*It|&ErOWS4v-)av${c=!qe=HZZ}I+|*RY5Sw`-N}|1G zpjlRl-w(JYPNZE4<2&=G_Vu7-Ylz~I`B*0wxFDe#BU`_;@9L;$x(YyL26C)N3US$d z%(i;{Is$v==u}A1ny?*9kW;Kv&#RG*DELP*(k1(sbX&FDUblau@_;lG>grP5L(r7OB7~52Q(x$m}DkD1I+R} z;4*0trbJb7;C8b^%U>A@til9M`nHOBb5e*Vr#8jm7BfSuds$)x@Rt5kZe4+Y7P;U? zyK1=RpIvS*jfX#d2c?m20$Ra!Ue0Y?fZ)$&j6A&OcfZy>+*bAfll{?1M+Lv_zV7&q zd%u+Z(MX^>{=^FQQro=$3;t*V$URdbVpFxgb%CF-?0yHO?H|Phn3H~~^(vv^n}<>> ztqS!jGN{rJ%J)!4Sgv3_yVd*>!NiclUUGewwk1~x*#gZjX^E)EB_g3oGd)B)s4#De zql|Q*MP80sTqn`%!XBQ-Y#Yq`6E~SRn@MdoyALQzp)70!-QbLn`}+1)z@DAu&a z!`9kwt^Nu@$6^{9*T!yVH(Ip$6ejm9EJgW{SGPkNM*!?aDax%qXQ0XOb-E2INMC`3) z_&)}{VhlOE3sy{OVng3<+uAji=*mR9w78=si)hY7x%A^RYdgjfJ(?(&es{<52gyXQ zCLP)D>gRA_XM-AHc(>v)i{W{5!N()V|3i$ZXl3!D01+dmpp=XDR{I8fm{-#$Y?m=> zTCf@S;RLw;Et3q2MXZ{~jJceDAAaLM1rtk;&1>5qRx_Zb8&wqJ+v6f~MzlT^Wlp&L zx<25=B%oC2;FJ*Pe4(JrPxu$7A^R&tfzwTgG6K@Z3Jae|0ypw>15$$uYfZ-iLK-eA zZcHaG`d8_pOct_G>_f*&v*IYCC6Y6cW2;le4qFZ*a3L!Is6xf}k9T{|lCo|gn?6@O316Y`s8*-xJh{gdZgpE>YiWp(-a4(j>Rp)%W!v}BjBU<-?g}@+ zUF90sl9qV_0jF)1;IGlxAfW9m2s z1DFsApxCjKQSue@fcdINhB{V|nqk=}R+`w8zwc5r zgX`d|3?YyJ9Z1M{iEN-sVgrhdFm(idU(REw=)_so4G2!Z#jp7O|mH~aF zH6}p^XdCh@B#>H5k(E`>oSfIs5s3F(ZCRU2Q*%kBNlxr)b1I69Ld^L+(s__K9FPa} zYK5I~{7~2bAOgjED1$tQ4s%Dig_aH6#!%)@+&AyVIzJd=&sJ?pIOYzvK!5fd!P_O? zZsb>W=(hMlKQpB36D~Ur^GA4(u$+SB@5vY#Ofjd;2Kp%l`ZtS3+3;UK9bu$^cF zKETu!29l)h7gB`u3Q-gxb5Pt%K?VECZ)!E>A8AjHe33r59v+(2l*p=aW>3DKm%?W< zGH!Ryx9l_sd_J~|80(1pk?YVXrIN=|N^4$YRX~(YfNd2<`9dpXhrETPhp4JcRBvkc zg}D-g>{DS<6wylVIQz1**CIbRl)&;Yna&d+?DU1|N{O%Oya)G|C`^iKEo?fL+#{i1 zLvAJs5k)ClIqXYL^v#B2A_v9GILQ#?%4g>XFkx+-sV1Pttz(7?3L&8gmUh;#tC@6( z&$yz=O&}!H5z5)UO>PtMgVRt21ejZ z@tD^jSQyHXB)&KNDq1%wN5jmI^BX&^&KW~wO4%AMDeka->(c9lGOJeaio0vtd*zCm zcvjk`!zRvKQNl|D=PE|~Mk`GnJ9dmA0AZTn|Io6^v&2P=CUUe~jIN<$mtQ)F$qR9% ze8q2ZT)+Kh3~}WI?cZd5UjGVy{y%DeeXY5-veqmk=4d}CvFsoaZJR>ZYE@ax@tenC zihrDmHha=-zkLZ3#=i@0Dfrj}azE>bSekF*-_x?TvTIGchgBWW)c>WtCX`?LUGLeY zkYOPZp9(hu1`U*$J|#YtjpEAe56}klMzg#xG#6WYDy4qNY+(#xe#5`1W%fM+8w8Ww z8t3j$=6u!80UT>O8n;JB>&Z}kkBYq7NBvS95kiSd$`0pFrwp=;mg7nQVnCh0rY%3< zkYfJN(RYutx0oI_!bR8Ok4Nvj@E)ue0r2gLG!@!7g*O_vnmtn5NMS3rdRn)bil_##8ue=t?MjIx>+ENLG|}3Mln- zlBc01uBXroa4b>CD{~@YeB5`i4^GgFm zm()^t&106TnhC7w#i`7vyCSk3iYZ^uUDx%c-v2Iu;% z(q)otbjWzVJ7&1xhTlQ^mPaL^mAjVvJp>~*s(s7sv}u^ zChZ!Bi>=M3G`J9DPDXS(LSd2im%zS(y(QyzH_2uOhtLKq1U z-h~VZBS08|FoF=?BV6I(oy=6my>a74N;T*GSLaj^ij27Td%xym0*oFmS)IIURhPPR zJQPS~;ZObhWz1DghY)(aml=9cl7R#8bhxu+pyUc#V^sKfPs-XJ^x-_TORi39gMK0E zAM(d88T6XbU35RVW^VhoZh%lJ1Ol$Bf2z zq|0s^u%r4tOfVpi`^mt0(@l16D3rjvHVr$>4><+xBUSBDBlLo45;Q-#W5Ke>F!bcB zU(yXvGrRip&>`ynHbyJ;v){rz8$L6SU*cNoqV+>N25kHN3Ibzf0rv~$B@;|!ZfHir zqzGIFknl;=GFy1KM$V#W4ck{SwQmn&bIj4q8xw$6-&qCo&AkB&7FV-E6LX}!tzLQ6 zn%{bhM>mb?kkR^=jjNF1xFy82v#xMyY|04sVkX_;TUf`gJa!3F#YwfjG~hU8 z*1B){AwPm%U20Ue>tC)|x5lZymPBbbm~f0g&TDPOHOTyC_u2h^PSX(#iJa*sykZe_ zJo5)E*s{0vBxBhCPzFkwoFXMzZK;f3z@uzler>1vm4GxGq##wZUH0q34dJG63pFyu z+;#@eN32jb%CJ<(apoTjV8($FIcxHhHJP6-Tx-Fyrpds0fIG+?;vPHE2%sFM!W`9t zJ0x{k;oR-nAz5piT%GUSm@=^)97IB-MLKV znl2lf`Rn2JMJ)_$4Pe}2%HJV1I`nrr_LS{B4y~x|No6=(Tf7sMv&mF|x@GOie^h_8 zo#bDwedqGtOFTNW(m_nb@rs%HnfA@T6o*4^+I>`Q> zZp-4w?f((AU@}1LbAYf&>JxdLFlR1$!8FsiyFY*CYqFsqHkoDx!Q9-^tEQOH6;+UU z^RII6KunEv40QsXqy$xlMKSH^@3&xG(@YS180cX$;<;Ay2$GSS7u+h(`Vgdt%aoNY8Ln~Bjh@82E1P%}vA;zE(8A_+ z`B~J>{ucjqnWJ`@*M!pEv6VW{vTqv71gs7Yj}jw#+xioRmzEBcGa8LEjS+-y|EB7O zG!%EV#|acIXEGI34N;CYv9MH~lPb*U`L2Tq2z5U@J^0`wv>+w}XGE)PcVZ;#A@LOn zkv~&=fPIEw%U=io3Kq%H8~MfHT0hQcm<(KKjjHh|Yw5MW!GcLDVSz7RILOEi3O4+m zrAoZg0A*s;h#9U#oJux=NTDW8yf+c%%|nr660pf0%<@Z@G0gKfUJZC1gPIvuKxQZRogoHd}A7A{u1NacgI0aFg<8P_%fT3?0$i{HMhX5!C%CxBDMI}Ead~LysvByCSt*m z+z+&3H>gl}e2~rE-Wh{HcvC@q#OsrUxt`|ocLuO_k&blmklnYT2a8Wrfb`VXXX(*u zAoBx2MsR+U^j(nsj2*Aw)o_iZ91D0*Q;-0Th0B%*Ca@UVU>@YK4czGwu1k_Q*eaex zNqHj87KsNBbMB#0XFA<#=)vr!exHC2)n9?n!(1fPE-V+mtr#v?4|ABNLUlqPGVtj! zE!PS{^4ZT%JYlerL^LxR;fBpy4KcxjD04HuZpT($ow4PGl+X-^;2hW7^>3K!Uj`Mh z#Qz_M4DiC_E#pB0iuB^f}v&FaQ;Lhmd&7h>AjAA^i8&$*=YNn@^YvW@o+NP1!o zTkb!#KC_+>@Uk0}y=K?nYc+<1*(F+kbVN*+v3oOe2z=od*AA{Y<-RV zvASu;E%__fLZ;-Q*NPIKC=Tue@7L!p<54*r_BAEorf9aPRs72iV~YkYySY7_%&oFM z0;5Aievrv;o{P?|B*3k+je;nMQgQ}Jejpi7fW6S%q{7YT;0i#EZ?XZE z1>9i_D|rNN;vas~WJbsw14rCKuI=(Z%p!;iKM~IV+}ny&=s3eP@%0~`%K2_| zzr8EDmc{@;1CcBv-?M!4fvSBb{Me)+X2?~V4EVgH{G^cR;(VAef^dzgM$Fkq^G&%s zye`xri;A^VE+9r-$oq-*R7s15PN1inP2FD!_ms#Q`F#c9HaO3ww zSKj9vi)us+u$&2EH9=sKIw@x%vVfZxd^d~5CE`+1<8?0%-C~tu)rBO4Y$67iOJEFK z0%On|?8o4}({cSgCZ9Q;S*y@A_-r2!!IT*O72>Z z8h+Mb1c_p^!vFf|hHZANqdq;FhILhH>507}3Ue7iYO!uJ6^kVz{;iePn3f#Hg13oj zxLl!Oj=Tuz9$JFG{V|PNHnN^++#29CiTU36eeuS1R7bINqTYV8CZ$X^^wAzpkSd@QXF79cdM%;>}R;j zUZ(4R2qW=wMF4YPrFGK8rgElD&QSwRh|`r9r#1Gv+#0(zSCx~UEqw;26`G1nKmFOy zM+uzKJyuA?yG$Sj6G-wbS#BynalxUXp}4y}0ZWgE*q&lrr72LQs!)db{S(5{+J@(> z4}uodRSzbE31mJL`D$E7?xoMM)N>BV!n72KpGy3hVwf86K@Ou1q+VncCOXaa^kYZ+ zR@sdpJ~Ol>nzliDxQw}dt~UV{lOK%BN?4zE(+=tn9^KsKz$8wxKrH5PkI_wK#@mj% zy5`YspqY@Nh4=)t9(e*ZeF>_cQIp$O-;4Qqj5bgPE#bf$St#vEBinnh-sotMqNes_ zqiG6omO2h`WsaQyUG$*O++M2ooCA6)s(9%JGm@u<@b;)qD7*jd@bR`uXaGBE$wGEl zBERPJ9~3`o5C=0Ti7=uJeHp?~YZ9=9N45EU0iVZ9!X4+;Bi$C%(wYpM`^>#&?MBX* zdBv|Fw#D=)B|HCDC7V#^OJSrmVDI31oQO;bmcb&XB`1L3nI0fMcdBy>!i9elunr|* zaqnWQ36?<%{>!Nu?Um(&EOu~b0(Ybb+ZKn@9tcF;43Kz$yAjjJ9p)CBmT1pS@Ni%^ z(@~;K#?*zAB6mHTh%-6ZM+&MEnS&+BBrXK$(e#sr z2Iq<+^_T)^Hi$cw{Mt$Jb-B=*bj7Q}2ojM!s^5a}6l7$a1G2;X{P8eJmnRIWRvhgi zXqnE_PbjwdhT1r6usapRJ$Aem0v0F|=f}N123D*6|4_%rfKWxI`yYYyQCS#@8zc&N z4~4hquDY;upK1riJg>N?BG2KyP+Atos*Ht3O9zff6hmY=78XJb2RQ+N;M$9zw0GKa z_b8YPL@KbIa^mu~-OZ-JDi!1%c8b-8Rb)C#&~gbQNS$e7x^9pwzFfuxs?^$AVX2en zUUuo~+a_#DPt*)y#dF)3&iw(*E`_FlwU=4E0cn#AgaT|ByizF)+iyEZ@q~gG3I!?v zQwW|t6fYe9mwj(BhNBSNHah@zmgpFq-v`seTE$09r)^`mts2(t5?co3%c$x6;kXPJ zg{sV}N{@(6o$r7twfv8N@uEQoDteLS6dmM zqFRBD)pYio_Q;(WwJjYu?}&HBd*X=sz8Gmdc~hZpE_$ue;Zxwr31mS8oyI8Vj9 zOU;;#b@NL(UuN(f>&{Q0@$VjXOz&ON^G9@LaM{Mbhe-U9n5R5~L`8V&|s$Hlq?xlv04W*Q2hT zh42q8eih0bTCRANh@C6(x%-g3iX(W$pRk>@g>|_}?Rj5-k$nM8Cxx0}tn6nNpX`oB zV~s3h`hP}XYbr_!!baS<;2jD9g^&$Y%b0ur7vKZ0=o7Vlu<_TQ5b@zIhc4|F?y@Vc@5fE!J_oZ#DHG$>!4PjY zNw7h6viLgq9cYU03ksRQLIg}SMLJprJ$u3L$k~ccnvgV?K7H1E7#Q__glhg=j}Nn3 z$^`E9aTG#|aAX1VOl{qNg&Vl-PE>`nL_ajisOq0f?fKfgm)nuC+%HCdv7v4kc~HMOUUncGxObXqSs7&dTYA;9RtCW4$Mz01#-LWr_rP>-Vx6hpG`_a2=!a~9U!6pMElmX0h^j>BQ2B64*$9KRPLcq( zNC+?X%bBasqe)O=@cZAULXH}kCnu4P{2zkRbC-=U10r=glT{~Du{OXQIEZAKMQJAR zTr_Vpg&m*QsJ=TY^o`K#qgQ|S3n%jgY=w;Bd66SCN?ri7G)xp4!eE@Hjb^1p#%lbr zu}vG;$7L8ekBwlJ2Pono_3Vqb%L3Lp%>?fGQW)W>6}2PIA6?x=PVS}vms&a?)u#H) z^LfiKRm|wpC2h>r>kR@nqs<2KTb*>DCL#}}{(0WE@mL*7%u}WBf~;_#>)$n*{R~VC z=Bl?W&Ev_D#f<~;eck}7H)n$h-E6cDo3Ju#&%P6}c!sHuMb48qx1jk`nlvp+21I&8 z(sfSi6boF3+5xQeWANVjIPwS_dlW=!?9T=hDD*@vd(I}KoIW*pqUYHEeM z9xMP0!6G2>aBe_tZOjcO|KX0?e%k??)(KvRTgsrfo%!Vnikhh_AtRfJM|vcP#P{Cn zz#)KRbjb2k3P;cwHskHdOjnUL#$d(0m9DM2(PT{8ROlJ44Xo+C-};B zI4&}J&jFc3szUG&epqTU9yA@2GvFaP10|CoX|?KPjkMbTdgfPRa4lELOr1n<|5*eI zg*%KFep_#Gt(9UFHup>JZSOaBo%AQQk^3xR<F- z^?LSuDjii0eCv?b|5*0p}ve9?)xDU_&8DE6FNKiBGNK&0rzQ9Q~96A_R28E@z3;oM{_L*FH=V=jBm?t)H2+9a{M=Bjpw=k|I=swS6!iC( z4qQbJ=6|p2B7b!@RFJ5SH2rr#F zZ`RQBV%=W#LGGA)hyCh%|0+avGSoE_8sE?9M+#&>ngGBF`*C~ZX-HS^sUW`HanW`P zoCd37wrB+T>)-WNG-Dz0hu~dYP<4D5S^l88d+9usCeH>FkbN#Sxg9F zmAq3raPH*@R2bjPmh72_N@c9n)qvo|GdtM*4;o!)BSgXLXDmAY1QmA^qg@)mYE275 ze8onSO$AGPdIfA_N4xZGUm?<$`ip3nimj~(o(6v?O^qVx!~MEVnTzs*BsIbqiyP(COG z5J1ZrJU6K3ntL`*RO(D_t;Iue=>K*y%b& z;k>iWLSfr1-N?%MY6c%*Mn(D1W20ZVac15%b>;|d!H-HKd$gJE4|lU@l~HXRi|i+> zkrACIA_a&zKI)YVEDs7+VndFpAWqXeM;}T@ehrI3Y9Ubg(YWS2dvBoX$U4dv)Zm-c z$HO-+^ zg!P_B1-?1>(n5qm{YNclN^YT%R5>%Y(7cfIQ&<_mEOF`e)Xksv7B*sOz$9R+*38B4 zXY}+2G*;Tq10I63!=L|$78N@2we$(ddTD>etFZQH_AC&x1#0Q#3}(`_5}Chd3n2Ik z8+{(Q#(eY8H4N!I6QpQWpNH+2>@5iSK_v50%p5vn_U_rsp|vs-cyORQ-W3hCC<(N? z@gz>k4pb?aOT;^4EnAA~_-fcz2vC4{iIQ^4%}@2mZb1gxE( z4r039TE!Hd>d7#6Z4=6=0Qw&bqZCpfcFZpU@q#7Q0R0*j!*-= znCA|yQ|q{U;jjwykv|Q7^&=kA?R*LRtUo^xqgHPoAhcMwnPg|DQxdCSa!bCdb{LY* zd2N1UmhE8c!gfi~#D`bjD{tT{;s4jY)(t_Hw)G)>vN?8EMRHf%aWU8b4UOPW2H z2xK~Ngmo#7dMc7ClY;MgCA7=iXjMa1F4tH$X1Qo zCfA=R6K}Io+sdHrQPSnbM{i+(f-K;!vD8}XEJXPuUT)3k_o-H5iNS0dqxn-w%fbov zQf&)9JD&#PYwUD}@!>nx{ksFP=yMXX0Iv!yHx{bsw0%E4tL`8p)A4b-hgo3Ab5;LSxx7c6g60 zAP7x1h(G38%dO*9^KG1lb3`fw&H#xkSR)0CcK*?$Z4ws!si7wO<+83cOq}++U~01^ zgcKXkD1e0X6UK_)IJL%#OB}EgVqM%c;3y>Y9Ao3!f?2aL$O^bgqbzs{$CyWioKyGcA!G$DQ=NalduLIB%RJmP02jP$vy@ zjln7l-Gl4%1k^YVL8fQ`o)cDYEyPmHIUuV;Fsf~I$fGIcUUi1zlqLg5l}YPmbeL~^ zb7U!+8-GE&XqB+%{C4D=4?mIK1ruxdyC5&daGME@X?yneJMj*>Gu&{(?>v8Y!Tl@s zCQLvm12|9fgZzLQyA0ISSZXf}SEv#3L)ufmbsnwUx(y$JQ!d(O9lW!AA4>RBq;z2G zvfzKL;IR2@^D1wGMi7(o?05N+b3ISdzMn+VSI zQim*B2*(e8IP0C+njNvM@t5+l#$TdH_%X=HF;RH=`hU9W{OA2w zdvIk1d$9arB{n$!KA3(!uFYY_9ScXDlur$tYw%oGJiOe6Of?Qf*`u>H1IQ5jB12Q6uM_mEb zpK0wZ7>uv)>%;&uAA+=@G5?)-yz%clk2n53`nEGca!vexODEICbTfKTLX8}!6e&QB z)t`dQt&V(Cfr&a_@{XvbX>DYrVrb)C726s4eB`XQLL%dy+B$00iqkm3S=kvBYktSd_T z_|5%?j&tgUr3E=*$ycp6mwXlddFm{io*a3Jj=$egewtX%_+PJYL3EfB>aEUmOIz!^ zP^AC0{`m2gOADTyte%K5KGJkI;{RvG@iTu^U-sA3qt1Tj!Gi#LZ0h;+Rv0=|a|0ck zKcZc96qG$btHcmPs&C0!pRqR5200(O}i*Z$g8T%Wifaj<)Q-n zqjSEO=+8}{UDPXdmYrXO_%6SoU3AP<(NS}h#fEc~i#BmIxP+$tm}`ZSg&Mej06p3~J(F>e~HaF}mm-+C@!Hd83@eVsw&J0P@^u z#jV}Nv>)621XuT^+-698;tQNZOZR(}%t8n-Z-+{e>0}1|1)Q?oStGy3H@N=qfr|&3 zI|q;DF{nVAqDW*PmT(p%d8O}SRwM`;_+`+v} zseqSSQPY{0i%#s@)Y4$q$XHRn&g!g}FMKP}^5p544Vv4#Za!kLQ9reE*t||&gCfWK zw2Kb>)2`av2w%|3jHkTVlu@w@S-DEm&%wvxD*hI8n<3p*#yBiUTwx~V_kyx~=Jl~t z2l3RJ@>@2V^I!TNve-gOrTw~SQxx;bthVhnY4>`fyEUopQATEg}z+#<94kddc|P*Fy&qkEW-b{Y3jMMgOJ2>g zt>}CIBmJ^#wnqmxHlw@Q>3i{174z)!qBUp-S;#Z($Zh8MsfXts8Wf%G#h(1<)BBrx zwTgYjMqF9x*!Aer+R0c?lKM#7)-mT7EWW{w)it>Gr*Hgy7a0t}OL?yOnVaqxZbo6# zO21c|bzk$1v*__Zq~+X;%a|7@R^MYZ3}VX7j#V;CpSAo9ou(52sOOKcM@JXxjn7JU zA^a-Y7a-53*#=Hf zdrXVl3f1*c6Vg!Fz#tN@mei^9W&Hr8J+_Y7KhK5ngHST z5C^y$F3N}jk$ro@QFnceUcx;Yh`;&I|7ABH8V3`o0Xnv; z`e(8qiNi8RNWebxqBk;TFM>fuPkADI5E6N9pl|Vqol6vx7OJ2sJgDPU(%h78QNbra zzSz!UXYglW`f1PjNw56sIOv~?&cig|IK_vt;)CD}I141fmkJBe;xtRyhOmNa_-Ffl z0iediWWWu2!XN;hCmHoOA9S?6&f(AiWqK5iE}b35WPOMi+e?{(4lh81(cgd{lcLc7 zK6guD;|EH80Mi?s3gR}}X-Bi3xt0N3ozlAaw)o1Js+dTK1Iu6m6Uc!sjN3&SE3-v8 zp@vepxY-r56_2hfUIe-NLe*{N>)7Kln z{H$69zkOw_x=@`^^aM>VrBUs;4JL~00ly|rQFZcVb2P)6Eom#(EWu+uu~cEa zTOme&G!zH+&xXY^GX|oTkK5P34*}@@6BUAHxwrHhi=9gbCOxE5T_%<5BW}FIz?CY# zAYdBHBjsijTcCKIC=34>WV{q2GMG`NFc<#Jw-xbo_$YwzJo~c#Mic7#jruuis3T(i z4)gvxRe>o4aWs%XxZKLZR`iuj(kCD@`ZKGh@WRR8`wk=8V+bs(8KluwHx?w6SIQ1` zdrtw*FlGaX<`tSc&J>xk&KZ442#fYTF=3yZwj|s`@B%7wrUfIZO6*6QY_s&X|Ck5s zfxQDJhSI;~yrKES?BlJesEa$345}oSo6KuHj(t6^%qXff&fL)&1VgSPcy76Kz*iukZ84duvq4m# z=^!SYBea=%6Gj)qHlFjJQ+N7ub4Gv);0 zicR=yNi2Y7SO77`-!#K5E0zRIa3Ru6Wu?!+)F&29$}D={Z|6BiEP# z(tZN-}_iT8ZdG6wk|15a?Nmh&cAJ`Z7m&d$3)=-ryu) zJMY22Sv~|ee2NKU9fi3MI=rsfkdf8#!6E zH-i>wqSljI$NHEIVlI-N7H9lSZ*!xFa>xk)+=@pB{efAq7Gfjr6A%CuSJ(5A@i3$h%IM36dm!>JQIHus(i;u~Y> zUbCdF&Sc=)&5}A+ymy|gj$0t%y*+5fytnh9Ss{M_Qfn}(2xn>u0^6D}tDn+IM7b1kH zthB$YzO{$Lrnfr3{Z{CJdte+I6np?u%VNr7qQXW%g}29CUMc@nO(0T%Z6CXqy~Cb0 zg~bg@1{v>W1+xbD5qS5gNvrRKo88f~PjLiGk{b@5jHJ_b8dJ8JpKZ-LAa!!@pj1A3y2~ywdI4{ z%Q8ld&!CF^9cJ1t4<%s$w9mk_ll+kBf@yr>T_hqr8^n{+#1Sfvm+3A4ErP7Si*+Z4 z0rvDrBPo4VO#y>J-!egL2e*aY%93;|qI8-H;;uW~G=q~WnF}mdok>G9$Hd~e3Tq{e zx=6}MMi+<_E;FrNmvQ`Tw$Vu>w0#JdAVr_hx~VqRm~^Q!zd{;OMVu5ynxW-GE$1|R zDBe34s-Xxn-^QJH$Wphl88avGB95~z{1HW z=}bdz2&%$F#pJUOhSj7#g zJrd5P)^+tHqBl4hIB(j*u>zT7$!PIOl=b zeYrv=rs&^o;k2CCMusQVWN@;MZAu# z;h(b6Pz%)WQwT*lM`dNrHLfHs|Moy2*NK*9CW!5~>K{zl^tZAPGMenb&ySV1c+ELY z^U(BW{5{md*bVKM%qZ(iC5O*N%**>%akx>Ik-OdLWLOr!PPWuYXCjo!(iAKVMbneJlSK3$Rmwgb0DCvqebjy*GN$#5j%G z;KCthed#9r6nqy!WEwV{NCS=|j&<>;lS0GUP!@N()zXfn zof1_vNpy9-lgV!{olM3)()&2U+-RzOcvkKp#F7;QJj`CxQ1c^;1}^71VEO;w0vpWO zWxBooDaiiDMlhoc8-XMTIom+nOmCK!rWc*y*xf7sJ4Vi!2*A3SwwTDeA8KV_J;aRF zXM*G?x^pKq#$U>xy02wI>aEfQxlb0No4@IF6tc4v+f36&_zV(m~&Kg^!!n)JP1m>$X-3Wtxa5~MAYjhuVA^^q7 zsF)~IOmK;;?JQ~a!ryY+FH{N`W^oS4!it0Fes!dE=Qzz~ZrjRVqtrNQ94cK8>@^#D z)HLz^;Llj(#_fl6NR%i*Gt1M37*p=EXqO6&3;I|sD$M9ovF{2RP{2viE|vDZK7deC zp#;oGxR5`2`i*LnE|)XWL=F>=mo>(#xBcIvUHZKv4a4ZtSkSxBrCS84P|6tT=$%pO^WZsw}rkfD46j4DBw#6MS zJX&7wg4FQXZG?R=ss}P+d0=^Hd1OI5;*b8SR1Gq%O}NE{ZDh2q7}8yRDu}=0xzAqp zR43LX)+W{^)+a7V>@szmB=KUaUGYO?v>bLDEwOHNQyipAArDIt(kPUFu5#;*rU`jm z>e#AVF`d77B@Ks?#1WQ9YI~m0ryi#*>s?dNih1ZHoemO7twYtKPM4(_=zj}7wiSA@ zSzS7150YitL}0t-!S)lW4?PN}-ps14J$1{kH7$OI%Fx+k%`tjX*7hJXLr?HLia4c^ zsGpYgrjZJDzK-p^luP2*-jEfnSHg}K9K zH2V1rOx8wVc|R!QPY3}lUfe_G!p+oe+;pGz*L^2 zuCaMxY&69QRHwHwh03d{|03rsGW`f?fSryxs_}2hh1Pb|(@aHb<%G7WP&;vTTcY&V>$`xz@L8;L73J zo$wx*6bg9J6oEpl0MKxL0s!F|W!MV0>)pLW0j!Ue4cuEj6yz#6cUbh}Naz!CVli>yLdYq3>J%h2%F=scUvdpyEm8H!zY2c4 zviS1&iue=is2M;%mdKJqHX&Acz~pQuSrJ4PBD1o9HQ8% zZiuT33mS}0RI-5khG&_6av;{gzv%bL;6IE*bzf(xNdxXnM-Ek?)bRV#yI{gW%OOjj z1%cX$)PpAGPZ!G{+IFCpe_?g>n$5BM42HL zLB_iQVkpqRh!|)g+87S67~dWjm#qG6`yzRY8tFaT$pUU{YKg@!(qQ!+kkaaoqM;_%9$=sO)rrg3 z(htBpM@{&;d2V@O8DpK~Kmj7mZ>p4Sh;1P;S30wWa>(RDuYjiZr0jzjajKw2SzeOG3b6SJgB*>G0K&0 zdi3Oy^-7|=UP(kDeG;(kGxwTN&pQ`dWS@)_h?7;p< zv9w|W>Ih|lxQL$jZgG!@>R-JUE~T`DamTL)k}-OJHi!=o>vdQr?$?cHev1hfjFSOU z<~UY3jLkMuf&G?8WBa1qW!rOPw2x9W%oD*2P3`KWII=@9*eq(n#9dTUwPQcbO+WIB zx3X_ewDKBxfkYg=rPZWZ%XDw(#(hSpj`r(*1&bjjHIe~b)p|ncA@iFRYb@AgDFq12 z;#C=(jrNgMiW9BSnILJUr5Df>UQh)amJG1`p+vC6p`KolX4w-j(kTH z0c2k`EDB(x(oEpi+GIqQV`{6Ms=-KG4>>VG)?Ar|xw1z4M5XZ`bQ#$?zB^i6c^!t@ z5uCkfSsGSWQn}mK{^Lznb@Jm4)l?f|(Hv4|bu$5gWeMp2lY_*NGv!J5fcupDw0m*P z6;SVv{DgeS4dy6FiQs*ZI^R@mDltV9Q>8#?6<(|HBGe`29#$|h`w2iSbEz$j`3ehb z5q*XoMc95-M>c(h3;f%oJ7cf`@i;MUL}w|y&MuoNGj!6~a`d`Cp{**0 zR)uvt!NyxZAZ+yoQ0HE27gp` z9HlajV47?I`6Qh!XGBx?Uk--4URtrZ^c^sHk8>Ps@}+Ig?x)_F=o4iD_k8CAu->}X zra8343$0vvzGBAoIVi&c?`@{Q?5;RCEUm9?1vTU|&v23+O>?BFCMMKb83)lU&6SM5h0IsKI zoxJqGVoU!b8RP3_0O^joLU?4lY_FDTqzIZl2To~1C?Fw>OWCiwSFRmkMo=C-Z?vX3 zYP;=;R7UDM*2T{cLHb^HY-X4;aar{x3#vP3gnd$)xy{V#SaRUn$(%U0+=8WB?}CZi z#9o)si;Ll$K)y;uPYuSq8V zFuCmMAmNg9kUPXtesmsW%4-4`UBZwy(#xG#b%{mE`UfEW+SsdtH0Cw*A?R@C9=?jn z!b}D6VP?r?`xX0D+^1mVtFQ|7h=6|0w^cxpvphe?8dKd3yy^mwX7) zua6^KluT*qxrE8KOa<|ga6#na{Qr1)yUC1cDNWS6OszOzDPN}2927?S6lCvn^oAwW z$9fL2e}PFxKD8lvi;kpCNt#YMSO$xjmYjeK*P0_Yp3W5nF>Qzo%fIMHvz|c6+5ddv zP~ysX1g4!YTa{b|b%LjX_)|RHLbQ56>piY^=|l7ay_;Dxzq;cL?lLtM#GguDY+YjA zZ6f=EWib1OPezR8OamtpGL@VNY{6ZiGMKpUJgWH|J61CdI20t!aL%{|O~eN`82>$p zO(>><_>uT>k%r7B=i5_30%4Yp`cwqBB+A>b57^}TfK7ut{M)}f4kB%kCl5^RD^-$h zfR(-g?;Yh0?HUZ*g8O_Wmi}V_wJn%R6}?z6XCzNuS&D&c(WiF@YLPVde+wQ@cRaOY zqh)ms-<${FoH95Y(-x@G~n+sQ3w1wi@xCAbzp0Xw9fCnY^VL}ac zC=sUqaNc_L7dxdnAUk|&jsu)c&K)y=w8^;IvN#@jDaf?Ex|x%mOKqs{3pr?)7JY(s*}F;sS5m zw`Tz^oIC?0>qSKom#LfSEKk)QXByr^0?DA%LqWV`xeG+SI6puY&{%`z) zGS(nU5bk+|JYEv#pPH-emuIWXBAk$+z6_Oy@dB)R@FWgP_M&>b`7MiAB3lc{g~L15 zJYQ^vv9KA&YK{UUG~^Jo!q?1VXDq!NxJkS|EZeZlvOydHCW<`ibIXD50H!&Y0bI+> zBtuzG#mod!B+lU*u9vOybz8NXl={Z1GoDKc+P>qoW z9N(~74jO6d9fiGn{avL@!Qyz9LH#c*o!z+hRt!Q}P_@WjntE?P<*?~RHgMk&BfU?| z?^r)9o(>Y9gBO4bLJ=Bby-QHD2}tjaQiLv>q6p4%e_YZx7fss6FV(VUvVZp$H#(81 z2Chq=u17SDkEJQV)#2XczVFdUHs?W`?1h>G!gzD!nU{9un9%v=0m2G*#3lfUKzF~N zM6pbv>VVuz@vTK25Fkwl$@r9S3)l*_f$d;aT!avCjTdSXg#Skp3!xP7sr}j1NHpmO zG(U2(=VBmDbZ7oI_}iks(TArxai^MeJ@QLeUq>0;5z4C>Kq{u>6qU0#wfjXB#y=kS zkdv=WA5jz4kq~7%c<3HY>Q5SWU25#J`C$y09%W7n@QDw;#dEwUR?{bjfF(> z=6ahThIq~f?lMn1&;yU!;7(>FS58n@tXmlINa8v;3)aB|9IFv>^{z8hun(lYY2-vw znSnGMRzFjdgPel`YEQ9Fhw0;m{)Jc3lA9xaA55>ZEi)}QHH}N%m*$6S7-;Ke?)UbH z=s=;3Foa6x)!_!0`b04iPIetdCR-<_Lf?AMRmUIU+bY`-(!hJ!iv9D@K-2)<`S`CI zE4u z>7AUG=MeQr=I1PNK9dzcLPVzl$I4_y(|jw3@%4EDM2eURQX);ELkW9hkicT3$EBq) zOat+2`E@+av!Fs|8+Rc&DwCej{fNDIXN?&f^wRWSa(@GB(eTO~yau25hsfL?Hs)Gk}CEd`8m2 z;U5&%Hgc$>iNI)A3C51wC;7F<(QIa#^bwfS&5mOa@sC+KHx+#=s%5BffO)eCGcYqr znIN{5E8}$M+_ImQD_%^8aC$fpV=)3fawpSQ**Fp721XIH*v3_LmO8Ofwr&Y(3F~YB z%(q9@iX9tNELZ0&TD43-{q#&Ad9C^f&;g9Hfcu3+lby_gW{BFebB+Bf@}$iV*8MN_ z_w&KRyd((~<`s${SE%77?8=#EGxVqz{jy0x{fxO%3J@-;a=RyGkl*i{hN8@LkZ>5# z(NyByB6H^f)FKScy7nTr3l%)G-_f}!fMrTysr0X!`QPiM(mOI0wG@+CX%L(+d#)UZv)~_jupy_oen4tLJ*UKA zkZo}gv)a3Tmi8Rkgg4f811)h?9Wzp)CesQ#9wpLOL#Dt&HQ0@WYH|vkEfm8yF&ECC z{U`O;ex;YvCqeNio4Ec$8A<8W{T+m|?^Qn==22k$KA5ppoj8m~d7&0g+*%N@st4%@ z<6?}H*a#00o+J#$^d;olCE@Bn1yD5`nmE~4a+-th*B5gzCyCGz`=_sQ)LvhSWz(zt z_fhdSQ6sM?Ry8nr4a1@a*)v)M8%NVu@r6&3diyj3`N$mG*^LEpSWE~(Umn^Y*&o{* zVzkf~a&2L-xy{xsR@~Q&l=U6Y=d z*vg{2$Og4Yf{wSiY}dir+QhKZj|rjX`@?#~bOHQ7+goJtIwMCFz#P@&?^LY_U8 z?`A5zHms8K<)sE!F*))Pbk&h?`WHhRLM?)SZ1>A{HClcTOw*R{QA>C+4>GV4Vut=R zK~nIfId2=XU9imu#Xu9la{~-4LhuY$iWeSxguj-=o`H0w5ZZC<(!K1Tezt`P+DNR>VT8SODzXrF;~n2skh9VG5@?RM>P9pm)w z>!d{W_rQ$v;*T+v;&tY+q+3lR&tVMKw;dJE4o16gJmG5=n=3B{uqc-Sq++KYrk@KI20Agq{v=>q zCACVcrCvwA>8VE&HxEp#_NjB{MEYCHyq+6CK*&D@t9k9P4CN+7KAdfUE)n}>_7*J4 zM(61l;Db#j1EaR6?~@dA$_DNlORc5OLbU7kUOZVwmK%euY(K&f%`;MVZhfz_S-~v* zsPFCDmuPDNA29Vo)k76RNVAF3_rUakVkmBkv2ge8I&eO4BvOnB)7>Tszk$m zW6kD8HgJD4lCLm0EarpdmL|)JZ|!U~Fz~+>6sWao;wbqPLyN1+xPya5b5Uh86C|IH zj{p<&b>Q{ZcX#>&qI%y9AT6>sSYHz(jRK+7o1-qxxzel<9;ARX0$`BkNSGLOfFY$v zaxJ&yE{q|X1{_zCNx{WL=!ZyfaWar@gS8w%1R-pU-UAbFv(fA8w|gfjyRSune~L*; z1eWJvivm*XB@H;vNb24Qw<2NYU{S!KMwIE{GN;7aW>uw((IQ^Wc#BifK{p#rpg3RP zk);5p7a(;Fr}P8x&Pq#*Byf_xpOb*K z7*E=6nwRH3bZSa%J5r_$T|y#irvT?pGkGHJV2|cscVJ+JnbAy9#o@b5`}_drjWHF( zM-oA@g>lKebI^={J`EGeK&P%6CPl$aDtnVy> z?U|8C1tB{rbp1CYEBo7PtJb^-NBk^z%pNV#II3?DWGxLWVtthU37fg-Fdyg)e zXop56{CSH{4YN)u32-k*{epI16Grl6T&0OBv9fGE(~_?aNW#sU2y`47PF-)cjg+W= zXI}8{RDkdnyOiF%wF^Tcj|YZ*r|J{`1>f({KBM;#7C!G{)E+`tXhiYK^V;*q^VS1X zyeYnv9DA-kH12>XJDxjE7I%5Vqjha|^o_zt$yb|1;;8LnHw79o6{~56I!D45X8&EU zmX(_h8EbO#+8tzJ%t)jAM;J!3XE!CfXcX(dH2Fv zCkl@rfppvj-spW?YjD?ZRN_$=HhRE5{F=Sle-$f{KL(kXV)Rr=3N4JpU`>kx{Nx{g zT&@B|${7Q3ukWqAvJ`dUl7cG=L%*PhCVMlXVR(4*M!v<;IaN0ru|D~?An+eItzO#6 zpzo9l9C<05;tp{~JrzlnNg<2IR9QvHhQN8oQy5Z>UgN$}Enw~|WGOq<(AqN}Tz+~M zC(vbq*lth2(&HgGRl6#AWwmTW;UW##cY{5k*Sg<2S|R7{W&1a&#*6_|xPn%J8DBI*1cMR3-|x3dg(HNhK2e+F>r-ZRwGKj?oW{Y)yWjg#BbQcx6W8Kom%}x ztL+cky)hW~MU|UVZBvBiKhN`Za_HDd0pg#&j#vu`EmF=_h*3~eb$WIDnb?4etqPd* zq6g%JW>muf3EmCev>&)}_#e?M!&rTUgaQzzQ?Gwgc4nBzU;XIQqD#8}wjh8;xqx=b z>VMofe;5<2p#0_wf?Pfi#fo&@YZZr!Sa}7~={8Z6{&aA;iNq9$FXaO&AwxFnEpG-| zEDbGtr=dI07Qs#P{;soe8n~&-l2TDHB}~_XXR5SV*vgZnQt`ikdv-00jmv3Po;l83 z87&y6f&R!=Pat<5`iv<+IA9qyk{35_V&nw@OrG4}Txd^yThWAhU(5o@`W6)`SlUY2 zt^rgPo27K%{Ep5Vs?haJ4P+sKYrTq9?N=*mmRnJ`D;3yx+1l84R;8Tj8-nB&ew)8t z#@sg2fU}$@>trnmZ#I~qxV4=rT?+TpfDejzr(NT`7<&4gd2&erQ%aZ~MR`=n|Jb(` zAv*+mSL$!N`&Nh;8z2j~BVF;A(klslu(#8@hw|;Hg`j#hQG?ysCmVdqDLt);&oAT)ap8L89-X>)*rE@@MFab zkrwN}&r^H_3-eP^RL=(S`6+9;b)3$Ikd)RTdQw``Za-Vr*u|i1fuM_(Lo*Dhf)LrRp{YcNGH==+QYMEGRJxF^Amww2dV_J z4XDiG?qf_nqJy=ER zS}iaSnlaFAQn#bp9mG8&5@r|gvN=ww%gE^FKPzFpms$#aN89D>^Iw(3zioa+2Ai$u;Gt$ZQ&*O}@u)ZaRdi%6aX$D9vvl|U{9vm^Z*Q+it{UIf;Kysu-Kr=*+ zD5`5a&AWbAkfUy-u*8~P-lwwP3|9n1w6|wO8$jw_Tr{w6F;lzn8ZrXOmNG$XK5f*o zaijZLnPO8v+01B)tK!ftRw-6XAeoWA52n|e>P+>f1>@tQ6hIc1L(F_84cHZ+K>;0V z5N=efZ>2osUS5=PF6n&In4>;YH9jPRgyYcE6#cwZ-O_17tNESKR1TpOi8(nR@7>!G zuc~&1D3tDH=H+I+53)iC1^I`CB(&bjpGwWV9C8$^0gGcN96n!c1s?VjAKm<8R6wbSaAvc9Y zFcWx+-1pg4PP*2s;FTFgJv8S^6M;42k`byX{^jBZWEv#1#svI^k88a74pih$0n%Ph zry#(4P5JHf&{gRmW-GUi+s^Iac5_WB#V%H<|Mzf zB&9T^&RTCZfVhybDwS^%QAG}&$rnrm&AWK4Zpa34`5g5rs^FLHK3UVt*(PikM#Kl= zG8c-&Ab{M?Q%5I08Vm!z?@y8YqtJ^XzR5xLx&InKYeydBaQlY>h2b2S`2V=uFEm-8~JtvNo z%1G}b6SylYTr4}#5ta?&YRxnbuyDzB zq-?nb#glQU)$JUZ)#O)%XtT_bR;j;QL7(Xh@WB>`V#1_>#>piqf+?@O0fgH9KA2X^ z|M(}jEJsYIZDSKZ;b1ABBP*?_(0+fE)zKul3^Dt!na!(i#$ZjrG~n0~TP;0Ge&c@Y zhH>7w;DcocysDu=@Q9swJv#G=RJ+Dh1a%NNu^Iwcjpn_f_RiWEn#T9O!q!$-vr_qo(o_(i$I{`eeFHQ07b{LoFAS7( zH7T_zq2WEaHPFhyVs-M6i0dyB>wWJv$Nu>$nB%K)*k)dFZ@YO_ovqCdzQeJAGURYM zYSm3zlg|jD-n2s@g#ID*PIQUR+qmI--5iQH5gHAW8li5uS4s}! z-%LG4NS8EzBwvjx^Q2k8Qx~N*C6>Ye`xR8_LV}@+3;+Ks>OnTW;cE83=tUwLRLng&b>6Am;|b)o1%+LliNQkOu1iKIf`Ts*ZgwE1W+WR^7f zV^BIZ0f71xRH@#&E9@^(zwp=6TreXHi5V|gxA37;X;tNv%0qtHKMEdHbE5ezkeJW! zgXu#Sy{LiV@mM1RSR-S{#CQ;1N6%y(JJyNG=uZXlVF0JDt)vU4;obk$9iC?XqT|SA z8(KT^cf@|=mKlN_?3OTnJA$J83CLWKvfZ}BHui@ofKKRxNM46fExdnAP*w))s1cM6 z;zt}Ej!ljS4l?L?p-`Z5OoWJz1x2P`s59+Y{WJy}HAD!7<%Wh<{H$je>sbu`IzeMm zr8B!p1CFB(tzJ(d{IU?8yUHhlNt9d0stg!a5L9^nZ#D zKr~DLex&b9Rol@?y=8+q?GpI4krKlvAS1e}s#>>>tY;dxBAA#tn0@r}V~>}%JZHYS zQ#cv@$!rjpXR|F#>9-MR5V08AU>-zfS=7GKd(ZA~N1o9C-B=a=!Ct1lE`YJqJ_M4KCF9Hov!T$uy$q%O9y{dFpLJ7CaZ`)G4Rt~z9c ze;%ykmv~n5`pG4PY=N3#J4X@2`eEFT^dlQqT2UvFs;~`(jk4FR@x!386iE%kDZqub z;v&Bwd%tfni-~vW?1b_mBiGBA1KDIfL`uwsj9jdyVKPgaZyao4F9ixSP(aC`x@ z&aNL2F!8SMf^3{J7Q=93b{2;Svf*X*3CJMLPYf8+WzQJzZiPqZPa1G6O3}vP zwgGMgHQ#0jFKKaWPrQ6M@flPQJU4^(%Js2iI!M&>%_3KyX6qD)c@xh7NwUSzKq}8! zQSQa&FVjKNI#V!Qo^*wE81LhLMMQULCP0K~k$P%@|yH`n<_bTa{!A%>`QSM2O<^Rjs zo5w|w<@us7BDaW>Ns=TxrV1#lOHo=?U3**ARndB zBs1ci-}x=yCD0QX&w7-zsL%J-)xkj1rgnjrYx5}87E?6Xh1{06t9k}wpx2PDjhf-k z81wcBL>ojhxsq2F3W4GYzUcwWs@4YST7GZ*O5AKvtF|2YsALCVgRS&AdlTEeaw$iE zQGn9H?;bTa<_IvUjz;=ghxceG4bZjcclZdKmFbk?zQLP7U7m8!s8p~y>o$mBM32_E|(tH2?YM- zZ8Y^V6ZF)r?AGB1Oem^rdP?N|X2r&SLL@K-=~p~$r3m@%q{Mer)kazn zs*6JSE*aSaq#JS*4>Q{iY}XT)Xd;n>wOkj8|x{^WuQc*&!ME`XOWw z!d`QN0rT?bUF*@e4+vIrrB+^Q%Rb!znvHQ%PdJ$U@DAog%N2&WIfH7=J~#B1b`S)_ zxo8TNS3X^1{-2B@NNFZbR_PmXPRYHBL3X@Tz%spE`wmV$HB#`|wPUTy4L^pf^`o<1 z-8b~_76{zdTgXAe!qo0C#TZ@BlcsJ9?X$;kTR-$8LeYUYCJbbLP7EjGk)=0$AnbJ5 z8nhOzLw3HPnZ*?z0?{&q26Z+$IcNDIsG^gQ{d}}a@}58EHsxn}SVE%52EgjTaj(*E zs2;KtNYs>Zin!7{@_{bTRsSiGDfkEL>b^_Wtbx|5S@O29e{NpPm+;aKP-@7G-%rCl zX@=gU-}SYiceoTv<+ANpE2eGx0eU9sal zG@6s!qTluEpBSGS<#GWNUTmp=(}*(;7Voe zlyPfOMG?n8ZDsmP!P<${s`gC(qh)`;iJ0T1Xh;&YvF@6Odg8FCrHbzBY8wlu6Q8Ao z`ONsP`vgO@^AeGj5Xhu6%o6;@N#q&HdLNg1h>_R~WP5E&CSCu zg{hDQ$|`ksP+WtOmo*E;0h3kTK$k08B%=sbCFFaJK1q*5>O#$x?R76?(jIL{gQ#EPPB@7U0tS=W?w!PF#8+{&gQXs^xoR_b6RqMm0d2mf!7K==GSe;L?7 zna6v;6geK1uKqNH@+|q5?<{^-w_>cv<=7~jn@EgKKV`t?tPu!54!SIx=@$dJYZ&5* z&PVb7_7Xr2G~ zbV}DY>*+GM+ltJ@PfJ2!VCW!aPD`J%&PyI*sx=qI?Y6Hp?6C`my>>|;Tw32;%I7bY zz;M3AE$mc?B;F%UA3cni$XqWta z4$g((^N#lo`Rfx^iOnt1Z{G_#R_}RFZ{-x|v@biT&#TCcg+2cx^sX4SG<=8~=0>2J8R6031PM#SmXA(0g$V{#+IcpEdvIc3aUf>@$X6a$S zOw#3g)FMc;ebE=_Z?R(rrFied0>-mMBn(=Ag?{L_Sn1c@W^xUAhI~V(vCvRtC^nQB zwEq7ILvwdi+8kw;~no!{YhVTZpYU-x21TF`2e0ui0z^vfNy_# zaN89#v~cFAWYy$0ds|cbodi}RhP!FwmLogw5;)N%5UTHzIMC6MV<9zY8S<`DDtEH%vi2SF!3fxxsSw(5-LdaQAxErzgyouZ~!_(y!{iD9`F^JjDBVzPHzNo zs!grh+*YCMTrm_1o%S>oDeGRsowMUHBi4I}Lp~eDkco_Z6Sc6n%4=f@CSDqf+7m)W zB6Eukxm#iP#=N(Ctw&fP( z8N0(iIB?e@XDOdl{~DCvRKT1n@We_TjxSJ#LX>M_vCCsz-Zwi~%*q}~SsP63hD(5cTTPUEm0@}H28rkB; z4m-44)3tp}_vm()-HIR@EOSxZiWu#~d2Tecd5b3;f?F^z1$N2d${6XheZZ?sK>5cW?Kz@G1THUQlNUWc8 zhdhP3QH*^C&6m@Zr^~Eb0H-to*#@kqt!Jz?7KqW?{?IQ$9T$2toR!fs!BDvf{14GP zmv#R;e2xtRtOq8z8S0R*xkZn6N&`QUrO8Ph{ac~JLugj1KBT0dac&chP&la+vX&j= zvUWHfkbJ5&#g>RBPn}?f+fDUn&OXtdxa+Y(n~3Jxb?GPhWiatK+f%e0hD;m29&|2( z+ElxkiOVcIX(l?Q(~;$V_#OmPCjo!v%9k3s@%D88Zr;IeH>zh;#@}aG#RE7b5rqc! zGR#j*YuuxHsN5%Em+{L7Ev0%QTKgDC#E&MipK_5U4PU%H2IsvxmYg3(`Nw7DAMW3v1x0mpd8VrN13G}$LUpRm*jYvl271I(JLh zrr5EVl4yA^cWB^8t(C$QDJo5(Pup(jd{Ho)XnRh4Cb-36H#>Sbc#w%GtlptyzM3xl zQ=?=a{*Cu)xnWU~I*SFPSPW8lzN^JQ_FU~~a0K)!6nfl28Y~$n-0^U_HQeTuwvIt( zN!+GSH}2UI4NdkG6uyFj`{makG760nFx}&WT9qS-eQ1F8NX5|Lw8<(;|zSIzD_Dzx3J5*@5HTWmZ1s=N;1*RgjH3qoD{o+EQ zQAqti2<`2YA^q0l))Ur~)>Bq+#|j;DN$yySg1KXM>jtvBb;K-y5Q1w6+hm4gu`B%a z_7Q@`=|J28?xxw-13>F>1LMjOBc!MB8JbHP&P?XsRx-ZRG<%Ha*}SW{Izk~+VNM`N z&W-%Iu+d4RXVVaJJ0SLT5Ab4>L}V2iWixK5MlWs?i4oLvWZ4}~5{9#}O#~fj8(kAR zGvw&idO}{CqJPafF(XPD(gL~^ppr?ig+ksrZRq5(2=c^*jT*><={vx?+2I`}q&aFd zcf#(ol2_hGsVMShG$Ha1WR6^H#95M<`vQtqyOQ4(uW(!7&o4I&zZUH9tAaS4s#Iu# z2$-2Llj(|w6h2+kIhWg`Kh(CwOtk5wv-LFc6ybzef@gMSdW=#!oTw7v-nho_v*|R)g>9oKXJieG+?Hn>g^Ed%n{m`Z} z_N}%S+cul`m1Q(z41>i~EnL$9TaWlGl0eTTA^RPRS|3=MbI+s+h^k77Pf+>=-Q;^= z%kfNCp%_BM}xe0l)Vyb9Cdd4VteGgpSx(p`-SW=mXJqQMFMgi zEBRHt+%F^-hx9uI9f1qh(1&>twfE`90urXW#ADHBj?_9r0g=2&^MkcUlTmIx8` zZ8ZJbM11R;U*=~ng7jiCMvd$5eR$w{=HJQkA~i`H@}gS&fh$ikC-M0qz2yD6T|Rn0 zzjweQl!`n#Y@%6xEoBb0)I+G{1Du6H!&b{-UNl}u?dA?MA?~HfGJ`qey#BoV6|}g` z8XWG}jjt1jy~Rw-28SGHE0qLjBi_dJ=lEK40i><{ zk#&t(NiuA0v3S->UV#GRJ(2fgDMBOT!jMwj3yP~zqO4=NjSjAtwWHI=|I5aU4Z8@c z_dWadHHbnKT+k3web>jr?Oo#Q((gpAxrYQuLOef4@ z&H}jMrX$M}6WKrAdtNt~BOTUn?4P1Jn;g${-|5aV&vmO9fvKCo{z@U^oP_M8dAZb3 zT!0JU8t`e9-l6z4uhqFb*OEDQbU;3F$0%D3)$KxiG&dB=jhVgK7ixs;%jONj>X5ET zZr(=INahc1blYs*e*luuWcuiK{oJaX&`q*?Ge?*bNjcp?W^nyZ3sIcLqYxNt`gYRC zFSnPoGAM3@{DT5z-%I0_<6r~zV=A_LX+%-mc^i!A=b$OSl+oT5V8bCHAQ=MLQgg=||2ir5uO5e8NP{!&oHW(S1^qw>4d)MQo; z>4ShG&Fjja0vPJ^U`wZN&1!&m{%)|PGuD@%Hft$8E0%idSA(KPLa?Qd8TB6W$ zSm^ZAU8SLNQEV-_U+zh5H5i%xOR%Me9a{+i=dXe;ZD=for9TX|)O755HDKW054P0k z$#w%6ZBDSI=2M$a!1;3oQ!&6P&8LxSU%2wNYl>JA}3!r@aG({KdwOH7> zx{<750+>s$4U%@8TP+E0;u&<^t}CktAqYp%5JksEbUg)Tp;whj(Nz^3W^tY8+%Q*t=l!78ih7=s2Ow0+;>ax*kXlL2VZ#_o2A}@V1 z_cG-A@D&4GM&nJbpHs(VE@tJ$?{oxfTa%FJcC0U4%flX-fo#L}!$z4t2KVxxo2v!F zA98MRp=Yj<&FRRJF3xYf2dI`)DOuK}t$ukq0A9rly;-unxK@NDCDTzuYexxyPS;lW zHvMeLlOC}BjIn1-@D=#8k8}rbw?{$%)4mzA<7#`mIC!$t5Xz0{VGpr~*gZdMjpPLAPH#uv@kyK(;N)tj2gK=Sk4K>O95n(NlZ4iIh(AfR1{U{BS6Md32Ui79 z=@ZVNMe!12cU~vq<2oC~D7CktOxbC@c4AcGAyzR2;^EII_^$WQA3lYM5EXQoafIKx zbI3xBVbW0Ka9k2nmF&}e&kX62rlY8U4TCZ{&c$p`p>zTCGqgg-J+FPO1cbsS z`%lpa6In(FF^}GczZ$F%;-p4LG}q$ba#%I%ZYB!b=4eK$5wXgKQlUr_cw$B7o9$hM z%ybe8bB5<3MfIr?h?+^WEW52gvv@B@Y!>-BDi>I159T4$qN+^&?&R{l^N23XbYwYX zydEmKaq=s1s!)VONqW2LQ@8?VVABXt?mvA4U;14RT!4u(Iu%7Um}ycgTk&D0F;|Es zG8~#<_2}Gf4I>=1&LX(&4t3-5&(UYW0Hl+{fb+ScQOFXYvUDKsU~J$e-oDScDg?rJ zZaT8~Sqj58zYKxLZ4&ZD1A#G>+l`Ov&z^5K62~h+O{`vS=zY|!m*qZc7$mpBe*Rf? zD0C?kkTnO%3j?M1Ikm`TH!Qmc)4H9E()~m(C*aCva}#jV-b&XQeemE8niSIP)QBGG%L zc~U52RkANTUN~e>5LAtZ*F`gk0o_a#RyIY2#VNy|K6QFbWOaYY7)YTBw9p~WRU(3{ ztKZR5;mrukvqT1Gi3IU49YtR>X@tu%Jk66O!YuYfl#*|rWT?&A8PZcJO-X$yufTZV zSa={?J$!^&?8^QGp1q1E>YAFh6R;a*?}C5>fPd(hkHcJ<0w^j(BmEAaT9k9ASzL@^ ziaQON7f&G$!nOfRPnLA|XHW}mv}^HsfOvk-`=%(FV6kD_p)Bd1{;B;3C_@4tOZ|nFKj97hXn!=h+IdY3fAt$0vMiGPT75Lu< zvpiN3VMNg}e9^K6ycPcx(1NNq6Tm{bZXr_#KZ!aIdT3tLly8Ekt4ZbSu%7R|LKuwD z`zxW#YpxlIS4w;B;nx-yz-ynPDZcR^&d#1tf;}=Ti0BG;KLa? zbLZl=H*5@nj3EU@v^g$VFIq2IFIzoUKmv@p|D-ZUG9K{B{jcCApkbDadfb2sH>)IM z?=`E-qx11gJ30w_+z(OmwMqKZ^*-V*JSzDWjAP$Iao2U4TNAav)DQP|j}dAw)0MES zD{jI>*Gtks0;rG~e ztXy(}P+52om2*&JWw(+H03LYr*KB3hg9e=Yl#Kx8fNc(n^EVX1Z$(xiOzCbO zF+wdp6WOjsUXN@=x1xqD0jzjIA^&u<$4U;~ZMS*9lp`R@l`1$|m@KtUXB;x_a_n~O zaoAA{?qiyB#aNPfBO0iRTH*&{_7w#ATLzj+)enPbqeKS9A>&@h7JE*Jo!`8V+5hxv zG_?5Uq4-L}C&5nz=fsRl8U-w}e~i*D7{&>9-k=)%NL{;Q6bq$B_BVk7ir!1U5#e5b zBZBBKMsZgx+;8ptjsa7Dz6Vmn8OYwo?c#QGd$?EOyHFM9@1@GAD{mnOX~J_=DOIHc zJW_;uE}O1CJi`+uH6`cF58LRfCm2YQK@q6{!hOs1FFS7{Fjny>q%=x~ASu9><}4Ae z+V@e?9aBJ=EofA^;eOYXC0*h%KBx}5$5t_4{qRSJLdOBwutk*SMzaD{Ed>?A)%=@S zfeCWm~`LE`?%hOGTM*5k&fC$x4Ak?(33cjKsvjD z*%Y!l#FqkbgO=76!#YT!5|H&|$U3f(gVEddK1KkqsfCgrpjyThiir-Uxw3+USOmf1 z?0NV4y~MlG3OYT8g7y823c8nnn_k#WDnUjd$X>&JkM)<-!l7KazC+ct>v~8JSS|wX`>vo?YBDB^pQ~T76lV+%=rq()8UTY;neLM;|8B!2Qyxp69y6YgG zvv2}A=JkfW0q6pLO*cF#(rwARfr!b>M<^{IiaK%fW-2ual8CeaX+BaZc&f}B6Om0S z!X?3;9$i=CBEbDjfNZ>1$W*j~PmIMr4^6+z-ebqu`zoC;Dec;4?%WOGE_U#@tDP~PJOkHKdPoPYXUawhW=JBk zR+*l#oWhcq&G2JlWq6N@E_yna>N*c{!caODwIRM$r-rVbsDVX2QEB8MQ=1EMX82TkN zqkVY+z|wy=o~Otir=)Wy#Jh7`HnK9L1CUfG*STm4g}w&=phOJub3;BMZzPZ#x$X>rSk>rH&qxz2D%kPy}k>zpZY^rmZ?Jt%1ko~@R zy3M#Xj%ZKR;%~$(4DE?ID2|kw-HGJ&P!86+vP1~K-hdjpH_~lxGDBoqHKxmaA)%wO z-WwDS(p43?KEAE_paFuyBwf1;;|DL)86amG7qyTBU=$Ee6?LhkOE+{_A{Poe`}=67 zx@@ItQQeU6iVkDqH!KZxxbN{Ue zJV8R5iEO8BN~-O)7K3;|B+Ra7AlvuUuTTLtpNFQsuqgF7W!C|}!NZn%GNYhknjH*_ z$t~lm``|8^?=3eemLw&X99N-@7?#ZND|ZHWtbPwA9dL9yx*QWC@8zZT6vW+RkBxV$ zl`6)cp!7Am50!OnkPfdr@AV^=d_iBS{gL-^<{pnuHc(k>n80YQc-eg1D5cH|X`rxN zti!ojM`Eu+K&QypX?W+!t=+6#)F9JL07u~HEAtc?oddMB_1O)JAiGXR_U)9u<73^L zQx`&r^fdul@0jIla?moNUCu}&c$adVHnw32eec)d^cG|Xjq-Br=0QVu)$kNZqta27 z-yLX0=oNGoH4z2$y#i)1$3wJlCxfE=@X3=)0%i6%w<%?iT zV}{B$!WVci*wWbT8;--L&hp0G`>`h!k+}vZ_7q6veTF?=g^w_u4HqO8UqNS|j?7(C z2#jzO-o&)#cp^zwd#kQZSBO-O%B#dsyb3ZNA3D3yOkFzc4japsRNsAbo7?D&US7wv zw#4LD2%*vj{{hNaVtUi5*m-wYEzllg=%>W={ffS^5JD}RI!D@&n0EtWsx?qH?qG|I zin4KAZ0(gHg2OfsON=n zD6+Ux67Th01at`FYqNM2YT)^kwUpUErE+h(9x`~<1Zz{%zxUL6W=jWU5TEFc1sFy*f9LCC`pFy4C-a@#EBiApO}-+HLu%DD=z z$4d0ilm*;dXo?>VsDT9BxsE;lcsK%(fJp(bUHLZ6w&jSJ5RiWh7>#A-U;X+Bt1{72 z`2>Yuvs6Y@MJ$PswP}^oB6Cq3g}5gzZ_n3P<#)zI`bQJpDooqv%90ANSnqGz8P^&Y zaA*_x3WN7qDM>L30i&84wtr+e93D+PvTWtaho^D^(VV~Ui9UX4l!Ix0DvBc838Wg& z|5bPPcZ7aQ3JQN1xl^5DR`#&Y6Ce<&*XldvCl=+`>B;He#JSn;H=fQUjeMsfloJ7_ zAWFaUWMn@dqDJEvw4v5zl03JI(KZj+8|JJNiAItGp>D?wG#+v}C)PJe^-7FVQAXYq z$eHcj;2{Q4Nyy&gP@PeJ(*W{-E=^c z4xf>z`ZIie3p?hL8cAb}5+$%$U^>6^5W^Xj^FrGcTqQE(d?}Pvk@BtCX&4?5c{#FI z-T(V_;qvqIrAYfH-rWU@pyRfcd3b&qpcSUTVN<8 zPs&ivxpf}60(_nQ!{(iw5_6NY5LZGsNpGKmOu4anu{q}dW#?-=7O}=-fvp*fxIN)Z z*rjYWtF$Sh49ClDWf$E%q)=a=w~y%p^GLhb5 zKR2&6Y?Qyo%SR>~zxL#cgkMP#b+4Pg*!Arp(m9hp2?#6RkjoRr7~Qj>_w$>Na;z|g zO4Ak~+iV=#eTBg8%~1+(%Quu#_B&NWhK1`5&tB^JN9pJqywPSyEcC`y%7F=L|KQUD zABrvvl1jgA!RuG}zxkQ~eD>Ehh%^3GTb=-^Tvx_0nQiN_{8F|1OlW zEr&KQgvT=vO&^^IIytt3sm>HYF4sSzQQ&GNAY)wqb%8i}0cGFSCW^AJ>EBNrzMvxj zGuna!T*Kc<~ zyHu(7n}ou!MFqlE`LHD*>xGz$G3%lLkXeTR;{}#Fd7S+-G<*0>q}t7^vrEHTJ3Wix zW_%k>dt?rdKU<6wg=~?N9(x9wnim|AU5qnCr}UyUBkUtj7D!yo99OfxV^FFC)H0%b z&`KyoYdIZmXZxQYuo7KR3JSyY;di^YLjQLv9csqcRO>QF;mX$gB(p*)rYw)HjaHKa zb8*~tEa*r?qrcg$9O~!k4y@j6C%)lVXuiMQ{)UI_PM@JUTZFzzfY2JKLzFURaXYrI zg}&7S?>ihPz*4l?eE(wBsfHE&BVqmtTk?TVGLu4#gpdyhv2Cci%w?yV!}O z^SPmf!&k?MOkP8XdD&z3?>kfyMh^9;qnI*`gd!23-rj0F5=14H z%}<(1+dm#C9wiy{wCZfJX+`F@9D#aGq%2Z0tb^zU5Q;f_f=b2oF?-^sD}-pzrE2nJcd{#PV!{L~iHhf&r09PWr9LTvnZJ50?*qLJjs z1)-(z+Xs!=r-$NXXL_GH5EmuC-cpoITzj*FHS9JsSFhD_M1nB|@q@f%OwnQ#CG@2q z%;)c2N&PKVpc$Xy^;iVkxa5k|8~3nc4m1ZD`$s5wbaJ~N?Yha=uU@x8Ci$dg(14ul z>mXiz7F@YbWe1r=mr6JFuvs+=;3Q2&$nTYZhyCl{+Bui8q0z}g>=>8eSW5XMuI=5L zQW%`u?a5NtT9H_k^B!<=(|NDS8wP)c6=YL6oBmbEmn!oZ`#UIcly>BTdrAET`Z8}q z4*TSI?E^+J4+D)P5m_sA8`*9agh3vjHPv|}@hy?He}?9)GjyZL{}r?xUtlgcCF&Wc zQZC~TBprS-7zvgCXJ~G=E=Tt!_9+{2-1T*yD6;M*+klUBPKyv00B_yvGd<0b@;e`t zPbD+MAFHx>c7VFKuO++acfXgbZ*XfPSDY%BdPZNUgbgD5$3bvk!`p`OTI#)lG064~AR`sQMZJ>X&{WUB1` zOL4JCxXox}eWM52Wv>oM(mKc<&AklsskP^b1`l}SFHpocrv(=;g7haD**Au&DhYnu z0E1Uz!X7RKMfid4%B)%`q&3SI2Y{+3$G9r|!M~4YAGCH`^K|(-|AF14556IloN~In z^{Spii7+n~`Y47fs=;M0f5%cpNOa6ZHd$ur+Y&{XY2PCfj(*VX07*5^{`SNZKBK9-MGnV|#CQV4DIQl^ zsznbWNT_k1H_kqO`21f;pT*keqiMI%9h4iYsTOa3JhqLiKa^W%Ad|a5+Wc*F;43J# zwo?NdtqPJhU)sxeY#4ID@z7)$b+;#QrCP;Mk_yDbA;Yo-8#DQ_W% zKlD^{^d8;cEPt@YBJ0sI(ZFd!I}%Uc*uW5HHK?@A{fvhGOoF+Zfu_<# zwTYBM*;h`YUm687_E)TngF!02AEMMZnSpF~bt~~!QlewGZtw}H}3!Uo?NNc^qFf!ZRTzoZy9eJm5!@rCS0%putRT; zx38(zBEDa{jcGfzmTa@eBI6N9by!W9+VLj^MI13#n$;3E*W8w|5&>);W(A{9nw&OV zun6!AGm))ArPkGLJlJ#GL?mX47EAdJeb4HSIJldFR=c_V3@(9bAUj^Ck}_dXr)K{M zmkX?{-Oy}R?UL@fwpz&P=If z3wF2jN6MpHS4--{K2X9nYlO-XWAMjfp@-m&XsEK(H`IWJz{eVM{oIHxx7Tve*K{Cs z)Un4L(0Nt((4?U7lQs&fEYHK>QjvH+lNK@KcMTw1j5(qcN_!2j)9}U1GXtOhjmFFB zNPqos%^wM!z{gDLKQ^xlO}m;jj@WP~lWfkyEug?cDlb zc*+E#6n7y0U%vVKz!j{7J!!hHDO|dyA#;FEJmwdB z;azwE&cqy%I#M1LUzxsR{iVH(oU2bc4&}4DFWwE4%`7Mnz9@ZfY0g(fAWT6~{SI&L zH)gnRkjc#}AY94F4lPH>YtLje2d)Yx67^^9wGq=B6^8uniJdc#gsd7KHIjHZT2QY_ zd))f?4@3aaP##`s#pTeb=j|EDaakYCNo?~|L)N|_B5nK-r93b_G)b6*VB-uWt&9HM zPtDF7zVSWcP-P$oO}+;nvO+PcWaZ7wn|%zHMkuZZZHWazom zfbf0^E9&#>jtIyx$v1yx!tU%P;m~JFLs34OL5ZP$XC+u=IN5KGQRi{RmRy1U48lmoI{$+bk4w8EFk2G7TmD@R9LB=+LIn z;T2#zfa4{clxCr6iXA<^L`U>s#)Z8V9}ZQb65v29)t}7iS=}5CSWJ3cJT&J@_dPr0 zF|$$Z61}Fl&)$#PHwpyoXIwj9E(8MEsZx+A1%*rfAab3`Dr|D-ACVh<*r9{({swYV z$3_@jjtLdy=<&XTO20$1CWSs)beUBPAl|=0ak_87h*x3+ zvonqT_j1}e=cHqj=gb;iu4N2bW2Wf2_)%FV-4ReN_SdEQeI`orHr0#b;rPqPHDAWZ}TS4kFZrJ-qW z7U7mKld4u(%RD~ily03rdcS|5?W|1yhYZK_PML_>R}(#YP4~L%o1G%uZ1Ko)+;JM6L4)WlI*0727jI%)3p`2EiUbsT*QQ8i zo$+G!A~=J7(1;AH45>h2KZ~mbqJQz$1ZG64gB0ABS*4IR1KA$i?y<#|8{t<&$GwF3 zG8`)AN%&sxMu^Hs1!ym?$_q`H#rg3t&C-ay&N&xra!)Cmrr7)uDF2ObqqpdEi06Kk3GZ$+-YadR2IG z=>K~5)sbr?g!m(r)-ie6$xdzztm%E(n>uRpIKAyo&QXZ2W{u+cbm~PUCcBG(A~*)-8PJ^6bWfp z3Sli!x+)dP)3mZUzG62swo>Ac3f?fLpkB(-38jujx~3R0f2xRaWsAg=Bu(qgb6o~^ zTalT_I%fp~;0>aj9U`HCD2KT{VnvvSnCZQxPdd&olQfdVpLsK4@2{Xzx*H6qb{s;) zccbsW`>A*8Jq56zjT)`j9%uRNqN5PLL$t#Wahq) z$t|Ho7H9|H!}h3X6eJWc3y_>SWKzT1bL*foECHJpT3$OYGki6w$@eRxU|vUnLNBJ;6iks+a7J z{;m~pcrC$F=LBeH|t(jn=kbccVQAR_mdW6GUHa zW>aoIbh{S?U0U(-oK%yDKMl4t#pTJBxVU0Qu%$Cz@4pO>DmmEFxW(6AfG>;Bw2r(n zWQ)04=Jomk9g&*^skZp8wl3g~{adi55ofDKJ)t!g%-8+d(G@Ud{p(;$t-I@&G)JNGnp zbN#MFBBNA{O^UGIRVLxuo5u&hG&6}x-Oy{ODCf%~LN(csGW~RK))`iYEl?RrpzI6m zda*rI`aoYaI<0|+y*cpb*Oz*x6SC>yc`@o%Il44Z$ao4cOerV!%`TYo)RRqk*`}h% zo1wQtFEK}$ql}C(T800-KU0GD`;bgWkUB%;zbn+`UTU`ky6qE`e$+UA`5rV)M5^W= z?$8N27}S&l1xdhwnO5>WeSiD5b6TmZASbWH#zUna76qiBM}NE6jL`s?Dpb`EB< zYJTw%<3Roqqnybf;49pU1u*K>mXE79OAV@p6XKz{QpaI2%LK1ESvgb*BxI-EOtXaF z%b@B+%qzOl0_ECN@9tBZVf5*jzk)K&$A7qB1CfAZGKqktI9eJiN7ZCp8BdQ!g_ z_l2Z28+r z)_hL_A=omZ8Kcr}Y#KsKBLTE^@2GzDt*$hHc~SGaK#2IS?t*I!(HfZTd%FtzX22aO zW?@R5QtnpRg~LPAQ=Ou1{`Tq-v*XUyIM~kLN3(~l`TEIE8wzn`f2K6b@xi?5(8Coo zPIh;jVf)aG8qGPLgOH5BKhOaQWo7*iUHjY|Ltk6F>ca&tK%s{rXP_@pX zT<&!$;?LT^l7CE(*Lesp&em6X5LlW%%9xTp>g{h%sC(&v_H|o4a+*)e^R0%;#J|durUH-}F~XS(V3| zD>;;kjZ3VJ9Ff?_ycM*&$fiGied?wh*I{20qQ+IbM1!YVavu>H?Y!=`VfjEyG|>}F z)9zRP;Lx7iWIO*2MkpJ;9&|2(2LGI399h1lMR)wNq-N#unI4>R3_8v_pd2a3KN2!A zyw+(51po$5V(t8c?M&nI*Y842;V*8t>WFKTpu#X!;sVIrXQ7zDXKk|tgpC77XN#_^ z4-+#_MKbt00{fD|3LGvLh!9=T%f`9YJv(7j``;$!k<6U&sZxe54q4-yZ=?oyQ*gmOvW`Qbmxy;A+}kf z+7Q_E-;NLFAY$sM_$x>EGRxc}VenJNWs9X<#a}q0uiA3xpW%DZsZSxfLXUySVS?D% z>6g6hke=>O!C~8`Nrn90@s3KC7(mPpW;AwNS8(SjPqfaHH99n=YNl+*0zg7)3ViN6 z&Q#}0J$9dNTf#ZVggN3WPpG)cLs(J#9DTCR&}jJIS4crQ&KySU5I>h9)rTnQykRno z2s19m8Ca&A=uIP{Da^8MW6bu;Yl!Sv;Y2nPQfYq}2hsC(P3&NWCzMnOSQB?5%-_oo zMB=@_&(9VZ+3sF-+HhNxvvmE7)J`8SY^w80M#BH)JLeQ==rRS*_{6JgC~j6J)zI z6eX{C2{y1jr$=B(@KXdSl|ypSRc}CvO`Chen8;M#4HBV#)1igWJo{YpyTOr=l6xznk4rK z`~&Z=@%QqT_gP=U;q^OY1ja~(P*&iOY~I7{8|#XO$TJ;TcAChD;}UH`z5tO#G2P29 zF$#S`AY~KTK)+dw$x?_}TqC_xk+sJm<6`8G+yiGBp-Rxhf79^q>l)R2>M#$DiZm_# zD6W^m`_GeJ_Eb@zCCq3+-4CTu^0BsbOBQDHvpk@h^zLMc=-frCyW%404X)X**vo9V zTgk}~P8uus>0UnBgNU<~sy!`zz?|w^1as9OV1P>2eRmb$v8}<@Zell`ejQ6xN}8~g zk)L(!mfovx1|~ssgjKVKED6(1?4y9MO z3rSKOnTc#Gln(9u6}Qe)CrudymW~fJH8Dla3jpMhL`5NUTiBJip#$??Ktz?w$x=dR zQc(E%2z4+=8}L@9Cr5~qHkP*g(2I*YVVPtw<&W9hqy1As1rgkZF@v3wzyxQH;hOy& z>VZ+Ca{z~tm(3K4F$Cn*rN(SG{OCw~26E&@oHL#`UNBPq?f7S&kHu%6kD&uG9a+4a zRvH0v*U~#jvv}C9<9miCXs}dw?Iu|2e4`9MoG5Sr`4{?`o1gH3oVjC zoXC3DTvmA2l;oSXKk50ttl6g;B*r#0<^z%HbGX7(P+diT!j~byP6ySF&X_yv8VLRU z1a;+NWa__zJTB2b74c6cJ>zEA@i3SkQIn!ib4qrlN_*-T-UMS@DPC(9-m`)cb1_S{H?^*ga*@fpEv5U| z7$n_a!c^twu>>7&oQ;n5<5$enJPF_Do0GP4L8tY*%R^k15TEfHLq1uA!kt z%r^`?gMrP@>@$(QO0O!ujY&DfpAkFISC;t)S;_`s1&{L5qVKWG zLY9XtjV`q)a`!UZ$+auv36M}CvYytKt6J8pbWp+r@Njhu%z&HrQ;sE zBV=btYe=95FKlG68!9zD4Xhi$cmKtWmab^xz6(Zw9;mC34B~N;_Ze54It^V0`7(w3 zc`^z=W~Iz!l=4YI%^C{yy-N@43E7rtu6+uZ8>eo+2e@LiVLP`r*_~TK@J`b;%W}@e z9&@*I1UP|C_Wq~K%S0bBn zl&lqFbUjZB(mwDP*-a;u>D-29um+78S0AG}Qt>PCG+c&56(|hC+8&;>$F%O$bpuox zng*5|qxzm+{TE_%oQ|TGn>E^&N(b*$6ak1YGUr)sx_-RcE@u}~X5z|pJVBkZ8;uXw z{fr=E&7}D?s7j9+$kwWQB?V1iiWL%+KO-OoPPZB4j_7-rcKkEJLZW6uapw;12`9Ke zn%WUp^fy+0XC|IVkojbl@m}FoL|iH<7|NkysJQxAB zaXboH%VpUKzjgJvfSv&vb!R%0h+fbv6r;2bgdp<6HJxiANQKx_w8D*%UGg-USeU9A zZH(quOTQv;K0z>E(_SqrAc-)VoyaWAh`=TGS=WSZF>Ey`x^N?I7P~P4#P`9c0k{NF zyw>WHn2dt=PFDLV8(CL-3ekK01f^2|Xqxy;rPKIaH04`M5Hp)RY#hw?=m=b%hOZc3 zj+wlACl94`#spdt{DU2g2P`~2OmOAowYv=cl~SXtnE_0qYP=xzf*!jKP7k)hlTSlY zdqSvP9qx7H(A@(@a$2W(D!%)-tn;E=ERmL4s1n>*WZYyK$G^uHVkdzVNQJie#SodO zj=;3<150&psMW3^P#%Q+XT96Q9D=5&>AqHhLDq z?KZwXwMGxuoEAdw(mx3~(k}Ec_IPA@8b>Id@Ta#KYPYSK1p`CP5l4@1WX+8*;wnl( zku4!Kxdc<2qesG{_!ak#n@*T$&I_`5z%|E??dmkiigx;d@R{Mz0f8agep&)G<|FNCQ=kDh+4#PQ`a_u=*wzouGEz>LXk+w z*`=aLe@~^C@8eyDZ*|hMT>Q(d#(S~wBB{D;%cpE`c1g+RA14|=*Cws$$~D7hj{6A+ zv0W5Dfxg7nx;`uP*T(~%TsaS`ywpR8^GxD>^j{|aM<_*zs*8I61D_RIa34Xv>e_V0dt$;&Q2P9d$xr+fwHBAD3o(6lG|;Nhv_N`g3mlVuYU`n8^KbK^`s zj8e5}-rQ2QyRD7@wx+732?CkDg6c-u=lM9yl_}^6@Q$yr?G2+8wwpa>d`XOk0G*Da z&e`ZH|0;2>SY#Y=_)Dn%27OPi2bk9Wtc|n;3A~#T{p-pq69KoIgW_O*pjK;JJH1F; z!;@14Iufu3e{m(-1HL?zIrt7IlT+830VCu;LGOxDOT&k_VQz%G!qw|m>l!9y*Uj%5 zU~x0VV%-GOOun{-9Ruv3uf(K|)l3Jcj!DvYf^H%=*Jb@iyj=((U!S(#(nLo& z70g7oz=OZyI5s|Utl7p01N#Za&AU{_p8`VOb$f*Oyx7#uNgH4;nsPHjC4dJTuNcP* znNq_mh6;TAvbyn{g^+{!VA4J&xm|4kc8NG8=;3Y>u!1&g(Q}z7x#5s^evVNC2tst?>HZ zLXLHYMuWCLPAyIzEc6W-CF3)C7M~l}SW^5Bk*1`fsH++(6Z`S^#Z}^T>8hQI(5Rz2 zrY6R(mRN${>351S$oX^h{bM@6eP6C2&ya6$anR0H{y)9FYrd4XH?;pr-95k-zKy0W z*0(U*m}31hmO7c>ilTOTItcZ-6uNJ(8r6}CB1)MuMwSHmHi@dAl1PCfn4F75iAU$R ziqKFsn7gfEgp+lR3CHG`3XrzSHrNB7`&@t;a(x=#9hGI)cNl! z`ZLRWKeH9wg*bY9@T-@vA4}5k_*dTQc#UnX4WRX^c~sD0gVF*{o0PwcTeGj7h5et7 zqU2KN5P}vr>+xy;PWwDfFYDWMq&6yF_LOn%lM;Wdx6eVbV^J>N*HTu65LDlEglR1m zlcl(pppv=S+HY}Q6=LDfzB6%piJpzjxv=NjAp%)92Tl2<%)(irp{g+5R(QnuZ&^u8 zVcVldxh%NWY|u2)T?@eGFYIrtRbdo2<*Bv0>^1E+_$JyNg=+oFvO^XE>Z#Eu*pZCO zpaH0n1FA;k3T{W;8QqcV!?BPF2L{v>*>S({J%4!U3L(Wi7fm@6*>4)s$y|>K(dn?8 z9X%YZPeu_%c5GCWfw!jgONfzvCa8#bhvDF^f+(WrLI=U|d7{aa<Vg0Als#yE?(d=>cs!4jqUn)=VY{gSa z^sKsb`J5eC`GhnUm_IR6n}1wjs@dK}*vq7#2!D021aExYU-n8*6#3H-x*rb8Dr?Y+ zO&4Q{@!m9qUg}@z#SlMl;?*WJT-)w|HiEWR?HGIM)ZQ5Bqk=c3On=z+qL(KybTg3c zao9dxa0l9CIyAf2;Lhtc$i}EDzBJdSe>u1@L`Fo=x+O!86g~$4!3W-?iy{v;ioWeW z1pa7zAgfVB!&HiKvrzQ&knJ2mj0muDp0idz<`REEf|X^E1W0T5d!~!~J<}oBrfHjK zm`)?HWD%U=X$aj7ZHW#bRac8)c(q8x?RaE4D6=cBJ;Q7+2MUgrerQhe-Gx^Yh+JeG zzrB=_mUBz~yQ{R`3_YWC6#XFVH76J_ullV@gXef;*&0r8@sJz4nZ_K+8qpTK&Uzfv zzRE+Wv}m*1J1v`Yw;-|^Kjj&F*>W14K~16RiJP(tofU*lwRFsYH+#&GN@~)M2md7J zA@t#qmK9%)c||jD6at<_v>0fbedKxtAP;!&Cyp~%WMW=N#GiTAdXhVbC?5CL8IbDG(^ zQr%#7khR*M9iBK)wDS0P9~?=iQrwS1Y~+vFiVDqt?(UG!bJ!F5fTriy z1yrEnwOeM&cR2&uZa9=&lmnv{TqtqX76ub--F!yhe|RJwUi^nBd7YsfO+s)Gox&&B zoa5B6rpLKQ!geXEXT@+yjQ*!j7g(z<{k)1mY!vi8u zTvyce^%8sO_24||^L$FY>x?w+`eNh5bM8!D}--sJt`Rdvc#xdv!Hw&6?+lgW5wdC7O{C7WM< z>i~<3C6`72a4?uWvFnHZFd7lq2?XiNN7yUt>O$dKn~aC(nLm261frk+7MfBTqBeq- z+8s63y3A2Y^$zRpG2Bgu4_@E5g2Br$VF;rM!Wz`CZyX}@yfxGMwb7XyVC-)Wh6j`9 zi%JXFL=<`|YE_slf)tAuAy-0j`v~V%3cYnD%#^}O&s>1BF)_WG1OjM%A6wD~9V5>E z7ILiM;0>wOb-x>Zzj=c=gQQ?Ln%i_I^&UN0g6$9Wp3h}b2)e6cnd0vpd3+OSssGJQjX zO|4{Jt1A>eS2bAj-R6yB2-gYsZm9}eLkNDI49Iv~kisNd5%>9F{we!;yB7OZ8j zQv#vui!L+;H>L?jTpU$RQ|D%QLjf|;ik|H`JUm4V=Z?jo%<>0dMH-b(sk+)|ez#lo zw&`-Qar~t>1>|Ge%6*W1uR^#0*>^;uHV{Rs*16V^wG4Y_UA#J&qx zmZ8w)KR0g}6jT%X?}u*G5!m+Bp7{A$`j&PhhCC|9qXTHbVavvjdLud-Rm6Eke-`T5 z_)!N266;JzDMU_mM1P+LHJyJRgr$Y^f#q4q5erg;DuY~Bfv6Pnxi>lVtJaBH`1z)u z%p;fuVgd*jrym1>yNDUcDPpkCd3$!-axii2r*)U^J;7v>=As*NW%v&_T*<#@z$3mM zfCEnKHWp@9p2756Q-FP+VW~=M=w8!;)@N|we%nhaPl14CGAlvKHF}OVin-7sxOk)?zDz&195?h|*@KVR%owpxk+T`GPf%s! z9!$c1A#l8od~Jes8Mlrr=kTR6&1>}$zU4z%dss}WOjG0k&_m)_XH5_~WKy+(6Q=Dy zhKt9QMdGd?aOtu~Clt_eqp=OW;cPl<@Jd|}#YM6WQ< zgtwiPGQ54!xT~L(OaW?-o-oG~Jh>>1HF#?>{hS^_W(U_3t$6~*PS;6?nn~CAMhDi@ znB`b(F%9p#DgN4@=yZo#h>7C|ZCJx+KCoPk#=~AVj2Lu^2!Dh((*$NqzPZh=>EBxp zDVG4TWL*`WmP?W7CR(qIIdxAGuqiEFVoNLCm4!;J73$-H%*B80YC+O-Mm=2FV>#Z3 zNT;d(j?PKLDK+oS<^i%?e5E72_x^CS$9N=K&Ol?kikYb$q7a2Zs81d238{K;#Dso} zOsBA*kAl?*bf2qdzx=9ig89nWww-JtTS*bk$*~EYbw)g_g^gND?7emc-5?#>`yh{S z6uoZKvwzPyJj|jVicBG{9-5qYA&2tuXuzE2Hi9$USx$L^=SCtcVzB3ZaHt7@ZY6Tg}exUa`Wru(%ea_aleoGW~40=TdT^*!i>^#GE&DQYYzC^lADGt(*~% zm$a!M!cDgk0RpGwY@zR)oTaM~5b9%`o-{l)P0O^Gw#H|UBmSyYFB{z+GPSR5v2?sw zt4Vc*d{K-bG@jz#05NxBo<>YZAoHS&7Rz}TL-YR4 zV)*+~-ouqXxTk-Y)r35wbj9k@OuGhausFRC*ikzMf6f_(yC>Fj7$ov0nAbp^p?XZ( z?Z`nG2cN+uGgogVX3O`s8V9DYMPa<8S=KkxL+8s_jKvcLOm}1aZ)BFoz-J-NLPGbr zqTfB3w;5og;kr?5lKv+Dc(|l>I4ks1)f2b*?%`h{x`?I-#BaBBO_;EfEfLs6js;Sb zCtjBqQrPZH05&wl5_o1+QQbQ{7>X7Hz)pnQZ_TU*gxemXv|>m#VDj>WGc%GwYE&tY4Z5-Z~hXkTmqsFTT6oc znUGT6TxKgA3&193Q!vl2HKvmsd#Q=w(Lh6B14@{uc>dGKrF9^%+3GD8YnU~R-w5Z9 zO-fvlSb=Gby(&ZnX2Gp8JQSaSTu}%v0vPVcFf7O3J4L_}h$-s&#xvL1%56_9ez8Ew z5Um`O{tigJ%6|OGgmLAEca5`at7XD@(@>yr9V)+eq7N)!wuZqxbafNA5-dDQ{uiGB z{y6zP`CyUEaG=`Cr>#P{#P`6mapT9I{OHpke>TS}D9WHkAYwD!NN+PW(ap4sib^+s zQ7-CHtw2nWKUUENaS}W8$`I0R_m+@Pj0((9^*i0{_@u3+?%z-W;UAx!vG$Ndt2B6W zp_afxopx2oley!Wb_}-Bj$BQqG^wfjBH-8))e$wx-eac>4-DA;Rg8+JYUkfow&zEV zF2s(QWDt76rXtdm4i`3}QiGYxtfA+!ZKQ8_5!%-A!0P=TxJ2aXr^Mw8bzvydd;pR+ z8GRdO*2hvb`dPHIBUCd$1F z-h?~Y-=D{feOd=dAwyhLL05re!%3MhIaHl5|qX|%oLr! zYafs_LBfU%om8*IV>@9vX+d)z%$ZnsVk3=yTB8xDU6ikmpN$&?B`RW|s z;_i-ro$jpp9X8~tsYD_ltV#s75l-&W@p97OdWvLj#rPl45|E3nNW<*^a&3jkZ^)p!4 z^)qyKtprIf^PK#7+Vjr8(R-|>63xrK3*!CG%EjSQ%k5jlZDJG+Bf%V^G0hVq{@q6) zb=v;Q@{Ib1x=cK$HWD79hNva#h&v|vwS9- ze|r94xezLbl$9W*pT0)noi5FLG&%6g>2l8k@i>aXA^5gk1Ydy)6`uv!RLpuBeRSDz za@sBG`F-jA=z#l-N{e6mwdDl1cwl=FbDDk_GeC)p7$-E9i-mxVdj}-?Q@Wj&6Kp_1 z*#Fi`E-p*%<^#*g(1DPPAq`*?xD>*%E@6eGjUKRDS;9|X{|IyM^ zzW|d?T&UH6>oJwNpF1S8Y8oqZ&C~m97GUM4-j90KzZ(rX{E|xr5tryedcdHz7!t0c zIF7OhNrcA7d50&aFzQ%W6hV_mz4}-?`j#Iv7)# z)LFWC@9Af(KFe8rR*w(mb5e?vYPG_&n4VV_hR*E?&^K;0p(pT(-cqmyjvz+z`?}Bd z%7#WHS-V7Usd@0)W`qYD~rJ5Xf8^T**4;%2^(C`P+k<>H^zd?$V4@c)ZW z-kBlA0i#Nx&_Wchzl0jpKp%Uk589I58zmhp4`9NNVv1rUQw)}l>JduP^i?03cdvrk zJVuSt9BLn8j(>G#o`j1T8c@f48IDRug+}1JAl_HNY+(Y1C~7=JrX38NZU-2k8xO2b z*#MoBYQK7f#wv)#Ai_6s32ma)Gb$tUVd4>&V8TUON5VB-$Ke}%xwubOd5<*(+$}%m zHG8&ut`oJ1U^SK`C?v#cE6r9 zqfeUk5BA2uY>0J*Q?%4lFB&^X>Li2_+x4jXWPeB?k=YD5ru+M^b9^VRZ_z4k7ER6a z>PtrgifmA#q#+%?nRL9ao^)WZ&q6Kh9T#!B7?FKEIxBOG0~tB^u%D)#Pp)ZOQ#vLE zsFRox-8EH&m_b$gK3+=M3|w?$?Bv$#e=4Clspk)-e;9834eDel)D;64tfuzvzF@?N z$(jkt?%?b#H;s}ehWZchcdTo9b|v|D}XgSbaG5$U0#QHhxk1@cc?k! z<-}JXNn)}DJ`AkOi?3^JXfv*fcjsp}p$Wo~*v8aJ?y2tVXD9{vH|>jVwNKo*w*Y&Tn(x{<+osbe z7D;cX`ZWt|4{Gai$V+p_$(y)3StWVrX%oWyek!&3lIF_9l|7$gba=g$Uq28Bu=zK% zH#~2DbpbAtCLvn4-f;Ml8)d5hM{g-z(K&)5&0p)j9{sKo^m-KQmSEKS&12}{|Fibb zdToDEb3GB;_bNOB@&4aEcNCE&f9*X1l^U&R`@t;q4SuF169>M*2s`H96QRDTq$_V* zPxtmkNH?nb%4+jVAuC3@{nS@ZEihj*zwMYcT)vBw!EZfjtq;4EpJn zZ#9aDxsPbB78rN^`e|L(C+MA@r9rzcc&PS<1LJ$V1?Io^{s(8M&6Kw0#l$HBri=X; zk2mgz=7w(!DKX204a&wy=Vgx>WlbM|BoCcOwUfI@N&cbCA2WH}ptx zb%e%B4SiGxG~8u(O%;EQDW}8&@Hhs!L}$S`=oF_yZ2B&oaIqkV-iT-9)h<<^=Di5mXQlf(2TN3Nz`SoVepkKsQGXw>}LH zlypT&+t^BSmkQnV*!rKoRK&vu(`7#*DVj?_s5!jxeDt^f~#^{;J3iMLDN?zopKsHIrr0}cN({bqk}O{X1~U6 zSw~E*5qq72uSYpfMNSLztC7;|82W%R$JNrL+oxNcmdY64NJ)6Td*WX|M zWb|n`em-fHvuDXoLw%tb?II3XuG;*xsWw}?jbrx+(fl4E8U?#0`g9VRl6VRTZ3|M8 zH`p{tz%11DCpC!QpYEx;fQA{Pxg-!G9i#Ncu}8$=gVUkdwB>wY&co*t>Zqa%*R5DV z@qMs703ghHFlD47q$6Jd;!8arSWekDgce1vi7bx1<&)H3T%PB#VY&)hJ>xS}M_Ydw zM&`xDN?Xkjz;T&i!;zr_Vm?CWC}F~ zp5&~PfKi^0t27K$$H+W}*Ugwa56Qg?;?u1e)=aB(j8LY=!PYh!Tsbu%lK70CxqQAN z%g~gbWspv-hy+OA$==15H8q-0we1ZWVVP0V8=UqjEJjoU^M8)j3CnSTP4$W3 zwWhJ1r3}`cPSQYQOh3u25>MK9IXm+tMrnAjIb7Py?5e59vN7LGnS>$kdmNXe-1~XV zS2kH)X>La;0ZVyI>vRc-d|>uFxvM0t<4x}fZU}JGA5ACP6}wuI0cnIfHUD00Tl;Mm z_1_l)djqM)%X~_Xk9CGi5I$|CjxuP5WzdYrQXTLR5_dDxOjwXmR4#$@^W;9_XkiT| ziLd}zzm9ptVU$-k16p_|r106FF`X(PT_E{%B#r5lvO!R{bhPB8ZR-tpELIMxOr!-W zm?q-op+{(pAzE+kI%8&YV>lMIlL7bCfY@jO_S)&~L|=)3PBzu z;ZlS1ByYiqNGcT6$Y+9=z7MlL>4fuky{@m0-OJ^!guI-vgP4fgA`6xZyOx4t(mO;414arQ%TK3 zx^dw^Mu{DrN^gO?zbV(UrUd=hPS~Q-X=jd z&O|G9xNG4L`79WMQ%kf$=^>(VvdWBA7^%SVj8u<)yT?pvmusRWh>6x{^8>@!D$M`uU;)vQCWK&mnvQ$aeQJt&Ks}@$QD=hA233Gf=FMj1`Qsl&!?X#|Oy#;v zVz^1b1kmNtIupiBL7tK-A+>Tdha*D~m%eA5chfl>58+Bpp%E{dF z?hurrs3zSnqh1$mup{I~C#qZ|+tM&(o6&(d4w4UryeJGD5It<<9`#TEq`O-%;Cnvo z6(f=T4E!D!Bp|Z0hXnT?NZL#8BR9}WKLMG3`eC9gCk`F=OF*Q&m5wcDM~`9x>Kqr& zJd0ueLPZ5276890s((}K7-__^&$cUp)bI0a4Zs>k) z6w9p?5H`Y%1)nmVmW_SNovGghaouDanNR7MpwPz+yc?7GiU+~dL9!$(9hzI6B5fv) zdqa)VEW7N)$62nx(c0=`NtakK*P%C0bmilM#5(8Hh9_+M<1K$ANw}5cUyq5}1I-k6 z>LdcLfmCRP(N3bL=r?HBt50y&6KxF#4boEt?>Pi&ri{OVm1va3px5sE@SJBN7~@7p z0@DkdFWxZjFJY#D4YKIGiS0yZ|0c6^QbOZRY$mR5L0T9*izrzq*;A~GtHS`9R_md| zwHV^1G3qmYm*~EVK!2lpp|&@8umv^YqSYnVsz_%^3le0t>zk`Rpw>6$%*THA+aOMB zd=F!e8!w!Dv<&;JIGH^>TUS7|bleP)kfBZj$%c_Tq<;VsyO=q6y_+jt6HH^BC!fFz zEA^azQy&XYdqU9;P&k&#WsW-AOwK;hPt-Rm6%oj>j)FssbamHeVrB$m@j#_j4HJ^V+{4Gf!1nI_#U9mjs!S$y>7>c5_VqAFp&{0wncSEE&xNp^}#{N2~ zVWT3}JU1QOwQWB3Sqp%**)|!1!!B+}dR#o38}?7c%OOXs=_=#P&X<`%F&F>YU6bWN z5Z7W5{?s~cRaJKFJVWHAcUw^jqusr8(ig8Bm2Fyyx{C-BEN?t>2sNiH-l(0FoU(9H z>PAz6QAxmOk8UCJggb%3OvrJcfH$42CND;l8&mDvN49Pjek$h2OF*OoAH!5Am{RB@ zC43TvY}-lmli^!N^ir<`$tvdRs@0)ldVZm1K%gI!Jendj`??hDIQz=jFk^8f4Iezc z^P8Oijo}&M`qxIa_1maZ6@%|c@DI+v{+pO)Y)&^Fj6L;_Oa>NeW=een7T769W*+li zpByv7GSSPi)u*3}AqJXkfaSBKS_;jrc@l{L_Ua~PcvP^XOe_(EJdbGwH}$8@V*pzi zB8E~vCcnsu`LoYrlY8}NMbG!gquyZ*U|+DbFoe$% zC2>}%ja(e1riR)q(kg9IMHMiMD(;7v$iu9Ud=RAXMuR6D_raWH)|e~5D0bJ1FQ3&L zC9CXyd04+Vaym5fE^&Z-m7a^1LVJCw&Ik_Mz@E5#u>^mMbSHKW22GxN`nT9&qvM`& zzu(E%(Tu_ZmKXGAl>|bb8Wc{DXPz1cYsEr+%0~=R6BB2jc97*6;;$+qX0JR?oj2Cz42PnI z+j8A}i**K{xm(swNRDOMsUc|}uy?q}=?Axw*e24>l(Wd*D?<*f2Gz6O+h&~Xy3S$Y z!vE@>wcLqTI8>*%5-?Z52vaG5n*s`|5PL7_T-L-j1NLPTRh&ZEtviW*y$ueOJk0~( zb@!G#xVWrZF%MT8LJ{n;*!P5>x?% zF7iafP8?8P0IV-f5=k=F1r0o!`;No8sM0>mnF?ZBqy~Jhrd-X`vZYxC3~Jsj@f}l@ zF@2Uim8PJaY%mXiLSs3o0F|H$sCx)2+sG@o6y6ahM)*JBmd)}G(vXB%acDE8fG`P= zfEISfcwDPcAYBL?M<~BVg-n%!zDT;tSE1Z}hEcGa6q_kxmi_&v^fUq^nk@j<0_#cB zV;ipFnOKCxYi!&v!1w&SS*n<;|9=WS-E8=2FBGrJXd@W?K@W>_LNHV$8H6Hm>=+RD zx$r+Ty8GkN27Cv^O$N`_1U!|u^UdDlHJeP>DUv)o$K*VpH+V_}%#kJ)I8H`N#mtdF zI@y~e9t?c}6vEeA3scFBpMOnwTF|vA?tPFTaVpf{Pn$iPj}XVVjx-}L%OrK5jmuPK z@l8@f_T3kz$#;}#4QoPlWdi9Ogl0(}S~ix9q#wz%nWI(=X?X|4cav?TVhTd`UDidW zpB6CHt0WMT7t;Yc!Cp|xh{vV}8eGtcO{Ts*D+}DbZaSSTZJEk7;OK4a38UWVH`{sJ zARq-zAGx(Z2BGL1*J`Yw@uci4)u5(WNsEogWD2?jM%Q-IyC zYbno?3sYaDjR&DDIT3_ZD_l&Nm(1g{lQasCssZgc#hDLG7?PWyawaN_qP`0jJ0rdY z^d~gYo-9#bd$oo=<2fp0ZGJ44=FSI}ZBciF#Iu(zfU$jU4~>;e5x9Jf%xZjDVF&Vy zvT@V7<-5RnjWP1o_it6r*!vd13kWPOE1#=*l&g#k7 z=B-W>5MzMxmPeaHn$1{92i?tUh^a>YT^u=sv1?-jHxM2rcn=Kx7!MWbJE;+S3dnui zMseeblo^v?thVrN$nB5G!pF7Y=!+%!79J;LZo>3M05ik`$25E1=@tmyEnrRHScxz( zlNRUnKnW{Pjg@GA@o+zyocWN82exAdzjw_m_BHFv$mLRzs&WrymdpJ);%oJVw;E8B z@C#H>a+i-;f9``*)@KhleU3UDZ-7{74XUVaaxX%nyxy}yeBMhz)J2nag(4T7{Qx9o z2XHUl0|RUsJ3?PhN@g?6NiFW?Go z-Na0u5Q+uNT{s>DA1AUjMlN|%EI*6{l$mq^TL%9}--4(f%Miel;u=#V;)+|$6cTEI3^u7yH zuaH;Cx!N0#j1f~A;vH`#<}eAdhR!{QzD=|@UoSJLw7cmOiGpaVE)ZMKtt|L1x4{s;1og23LHzy4jS;fqZyr`HDEQc+aQoVw- zH4m8g(71fb3Yf=Z1Bodh6jx>!Kd7;ofSHKW<=u=)`kPTqi4}5MDH=V+i}aAQ+l+1a z1;C0WK)$9y=9h3ikC7P?fXx{z4;q`=Kv9-R7XI8DmYl@W0!ll~WStB-xHtdb(0xh; zjx3iUl8^G_ zdxFHYD^5P%l4IC)BF`vkxQ0pLJ2A#&tZXhk%%c1z8H7GGt31Oh`2Tdcg&;9ceq%<8 zS5=e7_a*wtx`A^5m^*loJCTwqZ?lmKk+;S?d?&B*-UaD#yDS`CTE8wqP}T(YCP zpgPQ0X4rKAIkvs)%?|u^V>7Xpm@xa|B|S)FZd!$E#spxKfK$FNwec6%AKwZ_<#ar- zR@kLhjDt7m!LCsa!N*T(N;6xD4LpaeFANPO!UU)kvMk z1~^y>tzx$PeXy*#XF+<1`d+G_v_1s8Y#^VOrOE8JF|u&1+kx_0)q?ATJbd`o<6x8oC(q>ivZ8>= z{)zjPV898IV8VfZV;)a>HhN5w7MgN8RT0P*X_hPWk9SV(7R@C1pX-YexT!;Y7lC@M z3pLLtIoh`@K%=s$X~@%&nV0uPigH3e*)W@5Yx^CK*KReWL5)JGlU45rj_3WY-UObD*HrW&dq_p{pP(@%b(GorH2ZdhLJOK94 zsDoBGML72(7epUHGd;E{e0{ib%Ay>u;?u(PK^wGU z<<)Ldy)w#El+EZ50W#_%|w z^nGv0Oap-(sG$4OMMU^>4Iu;eauc295=hKlMbq_|zfJV*6fsnR(sZt1*8Ta$=@BzR zg8WBXzPZh=l77p1!byNbhZ0)$RVpk;7>3Ea_cgu1*#3H|s6;F@1%mqsEZpMo&+IN+ z?gnYVn~`f}$u9y0e(w^Y!Nz)u=t%|Pl6Mf#j@O4e zH=rULc;l#A7A+gLPXr-{%?gf=oZ0#YnAbpU(lz~gS|8a6Vti2ji!*qIK&>Rb{?oI!S0!#o;vwdk@oH>S?}h!1$Z7 zQqYKY+FKjm`5d-cibX-M<>K1{F0m+wOfzdKh^pyqUPOkv5 z`wY6ULlJNDWO)#OC>aeQ3Ne?hftNL7Bhme=YB{Rsl0j&rUB&nz4RXEEgFNO^e8ZoU zMtQEg-}<_;UJNBgDiRrYgGUD+m{Bb}4_FQw@f2uZh$m|`LNI43!nY^EQh|^>CUrSo zIYPb&IJR(2Tr<~WJs<9ylNL`*`F>jlVlqh`P?UDA@`%$uKErpNA#C>kWitJn7r#aK zTxV`tF7-1HSTbS{N8!Fs5;^xd4ThWuYr1+O4BJi%L70S^jvo7ZXyeNv@<-NCvE1Wm zs7Q~I67khlEQjqEE_UN0g_`=R50-3g9F0ZQRf?uOU;I$Rju4cyYb%^GcZiX-5B^>J z^dEXNkv3zyQNHdvarhjPb^1~-io~Yvfe2#z3%#WaD#p*D+WaSaOJnX-Y)3WEM|w*W zc~>_Qs$QYDbY5*<3({zwr&po(5*^p;5$7#f`}VJYztov&MMrZAu={1*6dl%&)=qWN zC}oTX)}0~Js8zkwblWu&gMj=uW)bxUjd>q$uia6Csp-4};$BD2#$q^-_xO#zo~{ju z#elU0sst0<%?IJ^Cx5V$CFsm+_V`F`kfbfBaYl@$9=VNlV4}Tejbt9sNv3;Np?kOp zgyAZP7;aYCNVz6FLFnfu0^4mWpOyds?A>pOwEST+ii)bVtd3H`)sL*Je6jO}RV}01 z^;;g{(jx4Ow`K#8(E4z&8?RsAoH^cZQ2qj1?Rd+!^^A{T<98{*jzh)B#J3dwqR+(x z@tGxq&}|NHN-axEHp2fi?G}Q?bxT0RE!*Ru^})Ao)xkHxEr4lZvc1&=p8+v{_vQLE zJX1&xyn1HCdg=>+bs|~{Z{C{(eMLNr#m&%Bt36D?alPC`32u=N>zhK>@>DLc-+f-5 zfQ{lU1yQZc4rZrD3-#*2_l05@DVBXHz@8hDV^LO8Y0$=_)6V-|Dt8p)gaj|LSyp*7 zYN@J>Z`-XeqG6|)+WLcgX6#7JHS2ZjkoBt=gf^^g{^O2yp81$oN<@ama@Nvk5z9h! zGhL8`XdvC2%c$#b(5G!M-27%%OtsY;PLEW zFZ6YbHLmY=s~20(jVBAbNtBGI0DD&KRjdCV30Be8jNK1E=gZ(c@gAwQdp&>5dpr_o zCPdl_@*!kh^mFH72R8Jx09X&(4;gsFR!g8Qdu$-FF-Ppf_)MGw3FGw`5o?pP*cWOH zqcA7wtd18PRxZ9j*RCGpBAcKS10)lHZLOlCG8NIb;Z-!2XKG2=)bm7TtD8eucd`Z- zc>4ee-4MNsm`dJI7$SQIj=hQRZyCB1D!CJR*rA86TPgs)Q~(m@2F8t{7roqZ7FRP$ zR8eYS&)%SVv&Bwl5y5>3QnJi*WDb5+Ve}MxOYLk$5G=@=se3#N4DAvHL0k%^b_#GW@DrA|0fCgLE9&AkI&`Vj}d7%p_l?w zS`&Jo!?=0CyfOHH;T%tC+M#e|%sWjxhae_6wE$TCwu@bcfHR|Ol?U}4d_M2`lj`^B znw(M*r7QKp6B3t3rtsg>Ux@F%pIPYtQ1r!K)>5r`;vQ)wlwbYmdBey9kEl;TDOc>i zMn8D$2}6B_nHAzsW6vCC+~=M1&^6RVwOfe84J}s5ctCJKPsnZbFEWv3)5%y$?qh7B zga7Gv$9P@+PtoW2WbMR%XCA%lY(j)DT3h{0ZUL1F98Ojt9PC{RZo*yyR=_uLlA#iM z{o&b`eB{dRqX{47j&Z0{D_0ka0ZdM8S&V;nuK|`2eQ7+0<+W;L8o0_M#=BKBEnGz_ zhB*-8tl=>mv{=86(F}J;0i9~-jAjZHpD9py%$PIp?&ZR@XA$+5^Mbf*-!6FpSR;i&$TJd(^Qr=B$K?b!43DUqA8|3J+>J_)-1WoxvFGL2L7_ zjT+xF%A(R_9q^4Q-wf&%2EAq16g=X7NRp_{6zsdJUM{W$jx@qWYvrs1##G+L~XIEn)n zzo=C}yuP)^()Z}*cTsJnnX+^wZRvB-i0%{vnB&HS^q>9OdzqX@0yZJeiNX7?w}J!X z(-pPkbfK8zP5I6v(Kp`rCTH`kkR}-Lt*6yhPp8TTTqHtF-vIMFs7~rQD|hz9m`To6 zP^6ebZ&|VMYM6^1M(SKR6 z;UiX*D7-N{IZnh{6qBbOBRbXZyj#*=*6j}36ZBP(?}%&oFK4Q^IZ!PZ0~k+K zVB=3V5jlfS3kqd&Ew7d7r57m)92A7)=6u(`Xec z03j_v-X^36NSG0a%IR3C+MAp%5eF*7 zJ$eq0glf?;_}0KxM7>$=4PeS~%a>$BW(`JxTnSSAyQPMHFr$j>f02|@0?cN1%ei3; z8%YI@Ys7WpG;xNI5p$J9Uv(X}*5FI~H-%sz$0Fd!v#CF2xw+kp6UB9-x)TKyR*CoG zZEvLXr_D+Lg=w-3m7;Eu&wnYb)2^yim{&x>3J>OOy%>ZS23MOmm}l$*a7oG#H0fu8P z{;FX3Y!v?IAU9SFTA&#Yl7lW$ikDG_HMywVx5)MGSc+cqSVvhEy?9 zz`J+d9%MJL0>pMPdg7Dp>!z5gjh-;^c>D+~`r+JNcjQ%7DsWt+{R-Qd}D8O@*dyT{Zwl_i)kXQ$o z3F*993=UeeqJ`6h^Pqr{ArsX49G8A4xIGv`xJ=GIkP_w!dBS-TF)WSV^T}U-w~(!5 zpaQ4=+EcR)C7VVy&8ml%bPsREV!fUYMq8^X?NnDJ`YoxzvB$dKrdDeY!n?$&BF_?W z|I7y#U%N^S745|=EPzPvZ?TkICFNUmnq74stjbsCR@H^Qx^gTPYx6Gxj=)@k0vTO2 zS=0c~%cvs>bLlowJm%sM-zgqghYX$61w#)P$lFi%u^H{%$#{oUbrf~_O&jghbR;@|8@4xYD(C8h-B?1oX`H6s?j8+r}ECsRMX0)YVH?dvUVHhGz zB?C?dn)~*1IkdoX5mNP=W@vTuUyy4nnvrLb$@|@E%}Vx;tjo6?#1sT6wBhj198Y=> z?y>6INC9kYBl%1bSMqo(AM*|ehW2mhp8xZ|kk@4^$v!_Efu;8EgM{nM9RBq3 zE!&~G$(;-;@)m;dLzx1`tP?f!Yg5cK+*w~(yoh#c3 zL-_$d28H9rI=oLhwZ9T`Up45HL`&&ILqkAP?I@5_<{S25<(+sbN+5xY*_P9x_*V8)#t`8Bf>+k;qdNR6GMQ1|BZ`=@3 zh*WN&CUhq^ReKL4`t#z=q`ooC*T%~9VNA~^8Q3Lt0v-Wz(NonI>*K3WUn0=@Q}3zX zWKt@evqz3odDmT$2%cUHA|_*Y8}OUS&4h^v9FJZ?OKv5?y}-x|cfIxEe7Ybd5X* zgBo7vvM5HX$p6sbdGk0L3}2vxqcUJnpVLFlQR^|=SoV;YY>ChUQ=zi=JN?$?TipzT zfb{CoV&ciylcbbcFt>iRLYjLHZV(TeLy8Wji`@|e_apG8la(N3b2;wJCtM{){yc0z zxaT|oUK(B*_#nTK>)G{Ss)9G8_iiC@G?0F^BDnyP(~wrb8pj*KXebifNZEM%zhk#& zsbAPsW=V-I)=cEKu0r5&DscGGD`g>az1UDt=8i@0+P5Jg)E*7udqf}GK7xA_%wNl1 zr=;$t3a7Q`!A!CA@g6CTNmG@Y(1N^~pqlhCW&gE$tQMPEQOLzK2A%J}gXvl%0WL7I z=Cq92+!p_voC%G6>7tzFYA!*Ic4PYSF#Jo592gClqS!!>*9Ac%a=No=nUEg)@%fTl zLFiHI3A|k+U+3l-uoG?NoDm1MeSm-)L@^Xd&BXw=V(g!9^<3z+p?1rMAi2}B*Rszt zdty;}%eKpX8c!@1gNROMFEhnHU^lQE*;dmI6XHTOw^=>8qNUU4-i%?NK?TBOYj+Th znI4SI=Z9LQO#oD}A0aMpO~-{@7W)FejJ%KLDE@B4sa1$d@D`X~&CE6^w8zxCSv0Qp z$r~iEiJj9;QjqRbHFZYVR}YMR`$VdZFI(2E1}b8kZdb%g_}`i>ziKNlp4GhJTaX;u z+l>5SkwnXr-W{$|#xOg+f9}-72o!&k04Ga%e9KxwP92yOi^R|MXZ{Zp#W}SY=D$eW z!KcS`Vao`{##1%d$e>=kGbrqaEzz_2rBnB|wW2}jm|0eTcclClnpm+0&Tv&H2>Pdk z+uQ^OhufLrSdQH>z%@EWfFy$Z4p@{y%}los6W^I-hn~3&gc|Wld5D0gQoiAqnlKwP z5#!p13L}6th&$^{C!Ti)1m$?6w{k6_74##p304wgX+Fb;Kpu>1nuEGK*3<6wby&MS zf@$@C#F2B9kT1q5KlMT)!_8h^XKy?uS_roi-qUeyEXr245b=xq7DPz*FJ5=u3_@9v zH(O<1+L7se9$djYA>yz{0@KEDPxxWLv1z=)B1#FWTXEv;IAeJB0ZJE%dC zs3t*CLiMFrS|t^bUjA$PJNMc+^qh1HzJt`ur~3g$g8deNd^XqO>Bh`Amw||jj!O=8 z4E^x!8tRDmKM0yhP!ngP9}@ki#5C?LZ;7L(*v+Ovq9{jv;XFR${DX|#=a!*NMxBi#MG%ry3EWJ}-(Jkv zv=qf`+3$kXYczgibiX)ghTDmbbRkT9$^HN=ZD$_Qvi7(RfmY6`)HIrG z4m;M;@&yK9_*{*}SvE1E3=y3rh#>vhEW@yJqQke?_VN~13J851uEK@v94AkdjD(`X zARYwokz==RXee@u?A;aW$}u^A>bz8saRF4N6R-YGs6jet1}+X*A4E%kwM^+&pDcA~ zicWvgH6I}ovw;4`H&b7&yBUl66`F(9=M7uVJhEfqOqI9-i3@9vik34x!^Hwn8jBri zvLO+LdY+sKtWb(uuB-cGK6bW(gjb!dh`^?9kpJ=8?sZ6T-l6_<9{-C_CXOWV)Xa!L z0UbzY#J0npQW94v7; z22_%&jziQ0^{zP+g4HoHbYaI{R;YtPC@b8h@l^}U0lguaNX`~F$ zZ--%hU7jeETD?6xj|b%L{AJJcO{Zv->U{^KUNz3(D#z$qbr>?Cp(sO4eZS;Zf|M+F zj*|8@D>Lq)n%AuUa(0s&9$tSR+w?QRL9SgXav(fJ*FnUAFwT_hxGEUB-P`i%>>J$) zc>9n;Ujdb!MC~_ki90XeTZL&k4M~5Zk_4;C1FbA04&^kxj}CX;3d>Cv)}R5|)p`}A z)8s5)jUboha+4LQ{QcLlmw4eig-PV3g79ii-A9KR`s~Pc#I0kIF~jCy+?e4qTXJD* z6(NidP^uPaKnglC?5EXH{^61QDZ#N=eyms zwA?M~utsPE%bKtY>z;&T?eRB2>^^RTaX;rV9^ljnM(oqJq^ZZkG8B-a0kZ*Y1ZTLj z9LL^mnm?8$n8jmame=}hKXLELy%clkb&nQhYprRV4I=7VwG<e-xlQDfSR|K z02->M5Y1Lp6SXIh9K4ZR0IX%PZRU0}x+i-EY=dc{3-$_8Jwavi7k&3TL|Ol!75k+t zLGtZ6HoZ^oBaRl@Rm>f`~~V6l0uVT@mR)BcB#wzK$;HDYl`qDOvTbWg3XP zhexBayQxYh?J4YJFdYen%1N8`V(%6PeOLmpO>i(B`;f*`WnGHWz<=cM7d}PzezE4^%ua9+@>>W@^VMIjAh&Z=uA~#L zlPIw>aqoawzu1G8&w=G-l>7d}B~HWZu>q3^*(!QYw?n$hJlMqJvRP)>j)(T0skGtf zcG!FB*3oJdAq``Fc6|@wJS2)i@CLmGLRCbLrz8%mHs;GD8S9LVyLP{ajito`_8-3d zn*eEIO%E}R#O#aIxfjLM=HsuzuzE{vQ9pHi%NjeztV;qRleW{u83G^G1zTV}QIyWx zajr3MRBbQIql79u;r+8k50;2u?YF*BTV@xdBKDIH-yL)c2%=pC9AiP_L4ko*{GBR- ztDsAy_xOCDl+IGVUE2*8z;0g_G4ouL0C&c^!`y0@o^O|%f~Fylbc|G`{N=`8+S658 zKfk+cdjx`mR)Cm?WxN z*lkj#nT-2=SRi0J}e9PvN_-cJ`-{8&4h)dwIN-dmF^h2(GB;XKTs_CWFu|K`O)ycZ_pAy1Blr z?yhZ|onY^XQBN3j4{{aV3iqXY8O{q?54Z1Md5 zl1tSQtdyILrf8(}%++7-UbC4)6r{N@>lN-GVo&buW8;pm#7Sg<^wv%KrxtUFs%}9~ zr_?7{ahTa2)Dq;Ubjzs=iE2P@E!$AB2VnJ@It$mnwtfN)?8YJFbuCmyTY337L$}@FtwW~cX^Nv)SFe=It}Dm_7bB~qs%ff9+JT=LtDd;mSEL@E^0MUTiXWno2bDP>>>X^re1pDEvw?NzwA?0}~HE>6- zM0YfaKIgt6g!0%6L_aY=TqG_LgTzVJ-L*A#bj+t{ z*q<_ezY?+pY*(ZL$3ap(n}672Y)Z>~6BP={>e3|4a7yc|+h+8tECAL67HKoS;j^PP z$2ru{_M&`<4v25{vTK-bdVcJ;F#1`KX-) zV(iRvmd;&~Xr#(iTRtplb#oQ5IU&W?wN`xo{`vvp>4iPZ(G9{3`ekSD#6T*3=%=H5OFbcYOYL~*iBOr4&82!~Fi#B8zWJzo)CTTxSbsDMLlw~8SIAq$OT9a% z9FI9mH1@x?U`SPhs*;v@pKGTGNqsrnj9dR-xlZ$#g@S^Bkv^ zR)^qETKDnG#UC^uSbqCIa9X|molFEF(q5YnHUIsMk}MkgeT%fy$m<|$^sqs~^HN&S z-JLWD)jWf^dW%8C9p9_1Rt2XUCwL^}wn~F?djDrfMg^=zkqB%eq9Pt#ZtJ0zt>GB2 zD_OnM&K&&r6WxXL(YBo9PFQDrxN*piF$)sa6s?$aN9G_Z1-(_Wqb2%zcx$x6E`S+m zBJ#vdAE90)C+!zH=Zhcucich^>1-yCIA1c%U=K10aHR&N#P+;bmDvQA+-n(b6xjw` zSs1lc0lC!8>nqKUL=Z|M2&r-?KrT7I%RjEFuZX>G%IbV7#G*Vd0oZba{SN3z;c24D zT2v$eF=k-u%!TG({nvnY#O|+%W0l7+w;c^d!}G@tMZ7x>y_6}y-fdJHsJS?>k}rY| z@x59BVzO;|6JAZ8FuqCDDe@U5*sFXN5^@6xeqd!$n%Fgn2*AHT!$~=4Z!ABx_5p** zU@rwx{_N%w=jSL~wBYghT9XlGK!bS0Vgc~Ca@K?uhbg`M#F>>AYPb0tAa9DHXtpVJ z|DGNTy20;*r1h3LP05}ZNQHUOCVu1+9oijI@;t(g1tU}- z2hvlS9^}mWEz+gk19V@Ofc`YbTRvT}%#=^`O#)`eq5Hi61mCmCM(|`t)|yfuZPX)T;zlLmy5zy+h}93NYmSOjXi_r5y90Jx z5w}UIqm&IpTU}zIprJGN`40E>rQ$2}WA#PzjP0m0CV10EJ-YMr0a}uRTVg;xl3J|Aq8aHJl3EQ5!Ec%#(H}HOmJ2-O4*>O_;ELvW86*y_smbzZx~7 z7HQfV(0|`mu^GW$DSAu)%Z~A47W?!B6;bK4?ET>iGxmElTpy}JZ{6#T#7>c6og*L2>6$A}mUp21c$wbPRhK$Ckg#~7o?hkmYV-Ed zSZ{0|0O?`ku(y(2qlwaO{ll}TS|d@!TMQxH)LH3&C#QOSlF_CK_*?5q!a}h|15lapGRpIF^E{!TwtlYL0CGrKHFR zIYuf(cpwS0;&5<7t20P-mhK@iX_#d7!5oDY&e8yd5Z47o%T-+PRQN~q2br(7p-pS! zz5`OF<%50+u^#?ouiImlu?RljP0EECnHz=z*EciA1 zGfou=m;(N{n2u`5A~PztFGhH#-yg&uB~%Pmm5%W5f`l%UG=Wil(=uR$03kJpM|Xl8 zs4OvJko*$8%8J>ttaPHwgkGXV5c0Kg#`FrsB00vApvFz5CqB*+t*%;qy29#e&~@Ni z_j5dnScDLCSz0)8h8)WmMNs>E(pPu18iD!O40SJ$rXoP~EinJ~Z2j@#wbbju>p%T= zP4z7=$-1-ypP@fm0b(ZU4?oi8Kz==Ng76f;WHG^g2gKhdRYoUdg9vifk;MTAIkkB< zN6$n%fz>Z5TJhLqe4U$s^;DLFXw~{I@cdh*$-?RGa8%x@*1;ckrU?thtNy8S7)gF+ z?bz)&wzlpEfy+_>WlHsleA-E_>0ONiO@f+LA$>ZMcdbHAvp2!K$7cP~qoUOSc|eB0 z1lt4)!eti#>-C7C2z*U0T-MI6m==fbQY--V;k6-wi)?~_*q1HNHph8? z0JJE6fNZQ@-L_@eI=C_YLku-c^%EIUx{hA)qhyo&D2bJeZ_a_VeF6;CG#C-KkPNs$ zIf{dLE^1R<55Oi4J_3t+nLMhU+(ph|L}%N8X+PqNL~w6{xNfqI%%^nzh`w#{h%&M~ zsjdku`;%VMU9k$iM9J#ni%jM!=MEmzv!5>~F;v4?zJRxi`P&i@wZWhbAuFVawZdGE z$H?a(?iazufuQxVlHhh^AYRCZ$b@H3o%8mK|E*|L_#s}-)`u%4G`8_eg)+f`X|>IL zQc4c|C>kmmt6@2$BqY31E3V}&R@HTN)ZXozQxL^?KCoQ1`SH6pTf2>8o8e-F+yAt+3{6#&Z{iV8q3mFL#CLYW}qKoabHa2o4UYCW^?^fbrqihd;!y#W)b; zy-JU{*F1BX`vXf`+dT2&Wpqdd>!wKORZ(^b>JjElM^1O32=j*b1(RsqiIhZ{(dIJU zw3yu@*!eAj9VfG@rrUH6#iO{(&VKM@@C3|5o08NV92fw+7;7M<`S~)^feIgRSM#dY zUe5~gzUv$IFs?Y51u^Do0{v?i!Fs4mD~Ev55D0l9bkoY(ryinl7QlW8p# zg;xWZ>6|eIMQj zv--c-B}U%Pd6Z4ute(ccnW~>+86@%_I?d?etItQj(>>%CQ(itwUALKshEhoau~bn{ zbl~*_T}Z?1RcPNY15v;K$1!4@sHYp~yF{H#=C&@6*xq_11l75!GPlkDl-=%R5q~RL zU3I52sA)}Q5SCTE1>(2*QefRZB{Nlda}E~Arz|kAZ>HPt+GcuFWltWe*W<(ZfO7cd(pmpjDj2C?(S*P>x9(bb8FGDcl3Lwa@1 za4%y10EI)s6~W=$-G!-!r|KA0#hkBQ|2|esr)ZEK6_YyifhEU=D?GZOy#%GYDx&cD zzZQ`0zfZm>XcLnihRNRUzD<_91f;~EV#Tj8bWjzo`id<1S}*rZxFwd(+Wq8>3pYZsJGn#`f2FSm@j0teuvnz+%ulx`3+~nE z^`ARMaXg#ssXv3^O|bxE2P4!v9oqZQP>;=f(r-5QfH_d+_YW+qcME9JIvOwo+|ijl zBcBfvGQ>fpRazES2hS1Z86FJW^3UMn!-NREErJH+xv zVAtGwo`_Y$i-1GU-7e+)hoI7j$eI_^0q!`4ErC~+hfF$j#i8u$55b38VwQ6exMZBr zP%lc(L~x&gx37k^SVw5wNymR`>gs6FPNI}wg$}}aVDrSvAP5FRqDVYoeU{so!V2vY z)nUII|9=d%U;i$;Sb!WfbDL%4Ao|Zw?YtI_O$99n(OFpKCI88wvbmE;%NOFsAH4ur zG1{VETu0XPjaPB8%eS4re9I&uXWnHe*?a7i;Xcc=YmGk76c53>JXrHQ5!n1))*iB# zobfwk`u^Zh?U6r{NR|a(2lvTlVk>brq%XwBtGm6{FkLFb!F_~wul(STsy9@8gL|H7 zX=4rkIQ?Yu(tAw$~Ms=?1jfX|+W;mSg<(^CUJ;@*YSSA{0aDN}Q^K zx4;S>n?_y?B8GrQ7lm1y(!B$5k=a12(_38D#Hvkw-&s)hEFDNn%z(~HOIV{LFflW6 ziW)=0h@Y$d#LE!{xldk^7)v5aCDgEYfb-DaZG>dNL}i)q3*qkhMiL7lD$mgMPJ2(? z_pz{*s8|Em4iJ-7o0=$zK#gO6kg5tHec$9PU5$E^i-F@-cp;@GZB=^b{ry)qttSvJ zwFE@2VV4$As_WT)_&z)YQ^l~e09ex^q^jACa3L%ab=FsL;Y^!@X?Jjp+fY}8sHH2o zl^`XX%Aw>9Y1L@qsr&)oE`ILk+}mKmUUDC~pFBW5A-*P_64QjrG~?^A;d_ToXj`O! z(4&t02*uT>bLpBpd?>mJ^MK{f%vjGP`R9MZnc|xGI7ulShxeWQUwhf|6_QWC`BRTu z%&q$f?{_{(VwOkheTGWB2xOYJO zxbfppCX6x*CK*C>Y2aVBW{J>h1Oi%K-6M+f#cFuUXO?_8+G9KtEi3Lij~lzQxbs5IPJ1&J_!D#txVuCf$ zA(V@9P(D!eL?m*nk}ofYD_Y&6Qg?)W0#rUUqp56E+eS_m@t8BFDvs6pTfR($cmShQN+DvH>}km{^3kR@_PagkY0W}maa-K2JO)=q0ApQjy7@mj_DWO8dgKcc z;XMlLV;lZI+WtH&s_o1l#eaL(wD+okVpGhZObX&~^r+`}h9sVoxs!BGcPE|nH~mda z($g>9ot&hf+v)DxeeWuYd5{8`6x3oy20=vyaoCIkqN0f6TsVP>it{Y@sVZu%wWoqS z_qqRg&hxNp*RH+Sde^(&_cKHaXkYs5q}@z^xg%8|nJn#&vKQ;s)5#7M0fQ$jQ+qeE z<(GI1e$vsv)S(kfCnP~Nk89dd+*iPo%NlndZ5}5uHWjL1DkS84SIrWCd^p_e8UmdN z#LT-svY=Y!JFJ%R5sr-x6M5n$n?2q-PI>g`-STK+_vg|`wF9O{@#J> zPi_^p1yUzP0rf_*fudr1v#Le^5De&V1MivoQ9bqCcIAC?vsDn#yBT}*T(%!R@x=o3 z8wuBaBV6aSLLJHV@aU4u`7Tc+_S~|uIKC>WDG#Mna2q1pJ5~4*G^-l*@-5!xFK*`E zC-lG*fW`ZfDnD{LO;%Imd0SWEk202Q!n|MEh{%SsJz9TY~KC@W`GttX%xZRVKkUvby4<0^l{W`{s#Q?KF z_ejpvGadvKsbzYUdq0^uzV{R1qS|4RG!l}bnTHV9+C&G`8>6~2=f)%O*ZIo6p5LKU zhphhinYx3xU^DUFUE!CxZ#n@E_lY1VD|ov<&i;u0Rhuc}#PyHxF*phsUWT0o=fD^^ z4=w8l z!~t_nq_`crcT~mOd7`xv2e>J99^)~VDl*N+UB(Ix*{Jl$$mYzc>#bFIOVhfh6uz|* zGEy*2#z&1Us0=t43tDXPq%6O5O8ffyV_a-^o2dyD`q=xof?Ea=dDcN;Ae@|n;O}O( zo}`J(EdfD&3OCv5*L$RwC@@9?(>}A187uC)$b#SEZ=5McjAs7tl!To%cj_tu@c?7w zh-}&~`ZlAKZnZBsa{s&gc$R+KXOL;hpte=n{P5W*QF$=az3xtOC5syki-09tj##Gs zUD@DzyZ}B0OXjW&qO$nde{4B?D*%5<3@~(Qh_Z(MWyu!m=HBWF!gN)pCX*A=mv<>g zMc|@k2EEolINopy7qI;0D>66CMK3O4Uklz}uG;dnggQ8J?z)<2I4qXA7|6(_dL}dP zlcp@uqMry3B>fq`uOgcvkF1iLFYl%=)J%SXPkYhQGSSK3v_E(okIN@A_uZaFHIL*C z>j>ez2*6}>s5n`{wZrlle6ETHMkf%UHSTVVogU-kah4bZ%xCpOqGo{zmXH7}WG^09 zsW%R*;8x7Cpt<#6*;CD*&=Fw^_v>LFwBV-u#~^`()%xzIhzwuWgXLZ}**w2q43Jp? zBCAy&`@jL~Hhp&*!D7DTqJSwy)1($ULa38b|AvvQ2{2X^HMni8C`uszBbhP#{j1Lu z-p3&@P;O9lz|-C3FYsg!kQ>yr;OL2=U239UBD-+XJ{Y^V>ngsMRyx%$o&HH%diPB= zZj#DgDVuGmNiU*^i7{MZ59PUss&%AxSD#{uyc?m>a>!_w7K+uDvzIfPUCc3 zbdjnF)tMtz6Qt>s>`$Ngx1r&#a+1s`6I!P2szZ00*WpvbLgAM+JJlzrvPvi%dt{#9 z*l+DOJD+Nacd=$pk2w8_=P#+6jw=%c$!a+WD+=n@bW&YZHzhS(%;z~rhJz(a$9X!& z_1K>TJg?}{;*3c|@pzP_&Aa3zgzh{oWels5W)N(?gXQy)dbUS+K!MxzbYa)k8$T4F zDP+}ytEj0{m(4_1$CqpI$kLSN9Yhi+QJ%VYO5I=56UnlL&XC1)V=D0It}r#WGfBH#LC0} zSLK^6?P}lGy0*A;+pa7%5m05XmHvP3 bJ5h;?)wJKr{-`_u@B50Vh*Lr4pbbZ$t zMGTV!%|}mHKOH#u*5E5lfx>gt$>NrB!ka?oiFUP8efMrZCvA~rMrC$TX_p*$TZFQs zruR^ z5cWUeaIIbD`=)Z~owX-xHNN+cnii_Iv?>6nuk(ofBPY*LLyw1EnsA_SBM*+LMTsOc z9+`R(ccn~HazT-L@1b!*k47T4eqUQ!UyUM?TX66tJx9To$^4P+3}$06-1{Kvn%Zll zq2#wL0)Wj5zMj&zY9yT{T{Xv}QWUS^J_d_>=V=pgpitu|Mmm{1#jT#(N{4+}80LKH zh{qpFQ1HNK^LffP1zB7SknB5GE^3mptrmRq3fMt*@2JV1%xeg}$bdSFjBeT9MR|wH zH8=rT=EJL;y||CMdHw8iyv!E?fAqrfH7}~)l;@j>^(4|ly>7f=yopq6%lBwu3Su{Y zG0WnV^?Gp6sf(Jv!3RM^XjAM|?a8P4R$hl6R)T^I0-FqJn2f0rZjA0dzZosm6>-H} z2`7Nn!o)hOLqJ9=(e-G0H9|K0*~AtEGjz~C$l3S)rn2aKG(I{m0YTNYtn`TP*HdRo z$vjU{w=GX|`SCkOQ0OqFMs|l0Y%1V@NmP2W!%VVT6~Nf3kg^w|NZ?S3Q@s60)je&U zXjZCSf#7L4xdsYg0!)Db72xDWhgEO-GI4hPm$C*A%-P53czaksmnGb(<5by+$^cxl zT_orD$fVQtg?2S@Fo=~?1Kjf6q&7&LprU}`qItg_El)6U_L-`&5}i}rpgOUt7L#hM z7{(ryu0_OM4!RN~^513C;DQ&>sUjr>Jo#b?Bfl_;gYR@7)8k#meO-yKFPj6y95$p( z8|9?V!=@iF${F)XD%-n&S^yL1>yNiYZZEa*-WOuMctjALrBo{wkTen-nl19eY;B&zW z5c*XAO#f2-O8s2_LO;R?N>n?)_%M{xm^=)nx7b-8lM?UEM6ejP%SgvfNYD0DdjNrmfn@O z$4F$nUj<8EQLm{t3La`AJ)xqEu{dYG^_ZIMTFZnLqn3pxBPl6 z@(CxwccRbf)B78lrmf}`Ee~*|e9hcS6X5i-Pf~9-loI8U)gbCHJ%?;8Gx2Zx2CB?>ip2o)Rdu^+yZH`- zb3JH=n5Rnnw*5WAUK~G1`Y$X1ASV>FNVc}&hMjrG8&l5n5zb|alIL?Zb@|-PzDhVOo!D+)V#V= z?d|f4c^*+>LiOt^6Mqhd;?}_e0RG`05xbS0yhrg(JmF&{xx$gl++?5Adj+?1s{bvG zN0_`J08tpSboB$pCPB$ptUS0~#gB9Z5OGIdd9afjIaR440`M%4ybs;dNQuLp!q_YO>cYLj)V4c2yR91wF$>u1#OIXUXmy*2Bw= zU_8M^*o}KzwWn=JqTiu%ZJizk;(TV@c+Kd=G_{Uu`80?nS$!p04-c3hzbIdhv(RNA z7V&aa@vBOU{f?pesPbKAdfoGggNTVN1# z!@&H4WA6=VT9T1H?ia2F>Xjn$y+HTFE7XfrXE^cI0S|&?911sOYIno&0Q?+%L3p-* zrSSA9B3u8MTRErHXZtO+(+2SmSO&r-Rmy-(QLR`$QDZk>*`^|ru;n0(bh;H23&fsy zp+IZb?ug6|ah+Jrb}PiF#{^e8_ir*g*xRjC<@HK`qiy!Iv+~G&6oCgeU+k_hrS; z9HD1+Z6hhEUT4wks)HJ(?!-=NCMA_7h|Tf9;$gdr^ZdQK+^T*JF8nS4;30N&Vt4e@ zIsIK0oc{Rm1+jAVZ)!dL<1=Q39`gKu8I6=QzhNH2J!%8jur@Ayg06qG3AnqIT zj%Xqq$6IiVYBJlg8cps+vW)T`u70eCBa-DMAn3X2g-Pzw-dUA&1kY^NtIQ-X*{=0J zllcvz9u_Zu&|^*-ZamKlBIZP_4C8N)uBNKaKwu^o0t2dGxJQV0l|_K_y8n?|)WQGk z%5Ppk_O3#^O$A1gM6VZ7FJn2cgu$z(>T?7=T-mZj^qzN{vBkl^yC1ejzN+?j1-4R# z=squ&x7K8iTau4p6-AS{2=;1|=HV3Tb2^meN?Nrr6V(-f>oNompS@nH%IfF}!bPTN zV8qxr_ZxVq+Vi?5n8;Ta0%MP=QjLdbxul0t#G76OES`5HN+h(u8?XB%25!pI)sIo| z*hRDO`N-1MlYp(xL155hff*-S-m6_(eD6w=8Mh21748$zXuD&L$1@=o_*Y4V#w40A z#?2OiXx@u*>gD32({O3S!Er|WeJQSUEw>3`0q2@L7isuj2Z)!)$viry}4RhKo@XS!8`&T&F9od9^(1^3GG8lIYuy-CvG^hrXB5r5#y`)Q9*wZmQR zm8*_1N$X)U4q|SqLb<0U!YqxYaDE5wlEUwNTF<20_atq?`|1N{tc3nnyn?VcbM)_9tW_p_*j8{= zk>m>@Ks#G+W~@-vf~vexbE(F>Zc{t55Lnqx5w;8>EA|aeTeq$W>;Z~|*6t_Y(xC(&AFqeMgEJuNoSIgbWp;W4~ z)j_Ds#7Hi~)^Akt+mQpZgXUs@pJxn~R%0f94>M2)&n?IkB(RGnlH-Ghlp|Q|pOYuB z<$bHT88t2Rjh^LJ_hk8nR)66X0A`U0SS==Ii&g+=pS$rTTvni63J(K z=PY;xZ&Jf$j*rkfi~vl1WKo2BPXi}l8|}=1@hD;b3@jtuhv(rWAmytif*@z8>>>xl zu!~O5bR>{t_ZZ0?5C?>l*D08fc`vjwC*uTnx}uxdnUtFP%wyg8mo2La+=wsWnb<30 z0Wjoh3g|+*i0%U2K*DBQ47j~A)U-zRvF!c;0ftL<+)nl3;-`RE=CN{scrnRw0RVRh zwf6ImsC~D3!*F8kAqR+pF3K`iRt+aHcouyCV#P#r+3EdmD4;IaVIY)aq%(BOZz}65 zjd-bfUS5uVk3}?aW90GSN*!6YmV|TWz>l+dD}#S^qGDm4O-rMS5A8rQ^dSgEL1>W> z@Cf5f-bJI3;v^#v@{K2j(?&w7A{v-#HF*K~0h@Ceyf8b?lwHr@_^o9Rw%_K^J)iH z&Tw~ZJQj*W+T&ua+dU`fYqySu+jPMnpTDAk=_M`fITXX6zesZGQ4IeoSbEU!kl$gy zsh~^DJZNc2Jng{GbUhaf{G2*b5wtqEy>d$!(VAHaBAphWRnWbAg7`dWBFU7}T7LWqI_|@N~wKYj$u=hU>`jE0y?Xp!XIbB3a$>;k?#c*9Ks|ko zol+H^+@U6()@+GUM65?RZOy*X)Nd2X@gl%w%hU=t#ZOG%U~y=U2IhfqvE180TpKDT zf62uGZjR3wyN0fD!f?yHO2?zf25_MAQxLFX*6ua4uJrtjZ%C5Y- zDj?pP#zHcy8V$IHu|#<{y!(4tg;GgR-&^A7)T?up><%NI(>f4$N_Sc(Yyip`h7Lih zW2k5`eiw`68P(c#X(PWPN`Z+W$hGy=Xt8}WN@?{kPTYwh3P7&cYm_Tmy;dE)`&>(0 zH4=ani#^o4Oiqmc{7kCTAJx+5l!VT#qhP&*=tnLFTwAc0c8hcQytC<4TMeqUyNN(j z!S73&iTz0=&(nuL$==IbFvcy~l^-4FkbkL|qqYzjb_Wn3O2+mjOmh--q$pt6%?dbb zS#)~y{iOoD&>3SJEhgiM3GH#fjEcwTciFWW`Ppj-B7tm)wK3Lqct5I+{Fwqs_ck}u zHL{2(qbr-t=uE@Cy%fQ!h?TiH-ICPny()sAA;L90EmUu>>|KQ8vEt;)SNFg!6R{?i zdBEI4?N1~r+(uhZI0;NzqGF|=aZneUhDo-@T#BljT4?JKFTA+tflKvuE>=;=?=U#d zjSlS-P=G2oQq65Q2nFZ_V5!iZS6@(HR10NNQimy+;~*5$V}M_qwq1Kb+o3(Eo&7bv z%L>QKwInpf?_PxHWOB1U5T-)xzZ(LyHpXKfFLKLR47dul)M8>gHB*wS5}XjtGn9Cg z*t%h{g1SSi_UNVwHBR(o&AwBdG@s%Wg0TmTrLqr73|0yUOgD2!U!$+pOV^YS8>K$e zx!mh&Ts)mM4W7~w1a~wznclk5t|KO>g!hnPxZ8qF%~z70M;W(|F)pbd2NBmvGsIm( zm-YO&njtbM@o23oISPLLZL)JET9AM1Yv?y>jt<77AM{;cL%-73S%>q}k9`eYRWa0H zA|N4&3X=p`=(mI2LiS_W4y^wCvHkD1Hwmvy&%-FwA)%C(ta3)`q4 zHIh*155}xWTGxy=tnQ>RMMOH%sLe@q%q{|@@mKkzm5UuK>(;As98UqgSj z*M0&A>s7vn{(^sG!_C2kzJ`9e<6#GKYUD$F4gL9Ke-n<$0m{$4HYnrrYvjJD>`^)S zv&xIr|41AkmiZF1sH57S95fNT%Q~=l%p%N_#?_;0IbTANy_}*j{HYk{?FJLB@%uCj zUE8zmvWj2_CJF}&?GC%C&dex0d)zYS7}19d(=R{BuCr=!A{&5U+%T)LE&c_Fv*nB@}3PM%IR;hH>n%(Z9}4MHH71ZXJldgcJ^xL00;vq(*92 zT2~QPz+DX%Ze`}EF~B~i&VgzxE$)zWo_lQ~n?xmWl|9vAAlMi3vypvzUft8^{1u)x z3BWQMF0%&$2ZS_CLXP)tSDJxc{ zHP}@{9eP<8j$h_d5aLP*EOLEZrt!xRYm>SW`=R$$kBpCvPmE=LTm5c;=f){Lg3b(Y zR`E#?Lv4eZ0yXmsj8L49^*o$oOJPt+9x7XW;fmhmO!NMC zyKebh0Q`@^qS+NJ7I+nXP(OWgN>%wNrCEj6@MmDzxxitaM<7ZLj3>GTICnukdcC2E zP{sTTNE8uNd|iOeg3DU0Jmw0n0a0glb3o2n2V@`*%{ck@YVg|?Loky@S66Rm2oTsJ zVD?mPiOd=tocz|*!|TZuuGcDAMO}-jY3tJBh4~?f^A>}S1^4xK%cu?l zzi=H-cSYhoa_qWpY!edJ^9V2s#gwGd>KTyblX*2!Q#N~rtuzv` zaY2&+k$>->*N%@M{1{Jj;eg$*Lx&C#fHoSKE`?kUsSU!l!HqD2w;@$Vx543C2}4C| z%&dqeHtUZ-+zGAhdJ<7h9v2*;&R12`;w^EZ=zPlzs+uY~2|9QzFpug)yA`u#ZO*A0 zqoBhnRC5kFHX22w?eXrOf#5LYexVZMl36!r!f63l_IAJ}d;xDEy-cJ)w@vihP22p5 zNUoSEiVcGmD(}0pRrc)bA$aC0kicI3Ma{NuJwC|)IrvmY6jl1K(!0{ZC>{lHNO2)d zaxim@=Gc8tDZT=u{@o}`Ap{l8#^OhG;Tc;P4!mnX)Kh8dB+7uE-P74yoJPFotOikS zCO47E$EPwIzMWp+3`0c<0!;rPgnaIKKe5$42*tt0({b=}M2WrBJwuLZ>eG&C1btQ+ z2eIlpI=ilm!!OBGDqG~giG5b`Ze=R;-3Y594{+R!|HI{snVOZk-RC|Gy z!i8&D{i4Fe2aaBV_YZ}bx_v`>(M>;`aL39y{yFP)j&vSVi2elg@w^h++J}XB-I53Q zRol2Hf%!qNLSCDtn1eVORa~}uKx3~!)oZsnElSv*`rVh9UDyZLIU8Us zdN=V(;H}30EPY&I#1)yZfn_zCZ~qBu%at%`O<;fj-+RNeyKQ^8#>J<|CZ5$l)wP#0n@CMNBJ1b1f{ zVqVE<<^F5ljeMMQT3iobo5R z5HeyoW5@u9SgH3=PefBs658g&xiFwh4NVJm8y)0AKR%7up&UO-3`5@^cyr`TEP>aO zGjTqluI?BMC+2vx6lW-wc9$cxSJN%EW4KRZa6RJ7IvsXpBvk|$|xey^w~}v zqBG7MA<38T%~|geh0S&I_ORRzx&Bh8A3DQF15<0Hu%-GO{*mwCKS6JJAuzOtkNThW zuLd=s7I16^jI&#LHJRdK?f?QIP*-^?q4l>I1b8RY%U~bX%W%Msu?To9YIZJ0@F@{H zBcF95=#tyVOX@rH!_FIru{CQbm#FL*!=^zUZ{xA`X(-TU%yty=;ddNKkR`1zL@t{| z{wp?IPgL%AFtzNCQ{yxVtqOLtIAz9a|GL?^o@}Da@(i`#Pb-bW#h8V_@K69<5%UsO z%_WG$mOuk6frL-o3J@j(pFkODfn5W4ATG4Wzqd#&w`*+sue!-fKYT9>fuT;{u6h#i zG{6I$;723U)181U5y-j~Gdcg1Jc$p23~4rZH}sul#sUMafgD=^&3rcES${Ey|0Yvl zskdbUykE)y!w$05GuO}#x!Nn+@lcAqDPd+Rs|><(eUC~Cy^z$SmU^aav0Cb^%{HiBF*`wqwJqZ^YP zT9QcbfqzIGdl$^q9$Y{~NWzO{pPmp&W3Mz&uZ!LhW;9;|%g^XY?Sr|pYO$8r94C<7 z^vfdyEKwXMcD9zVe2vG(^jtl~ThTVU!FS9qp(giZ2#ALRN}`=6-cpl8jo%{+9EuD& z(4VSqXx?>{#L|k&<^yWp7KPz8zLXp$|L?SO_kV$-gRd_x8`NbF)Hmyixwr^$&EdI> zNa~k8SFdToIYtQgF-Vx0x6r$Yi@F{xuhdj&s^{To??zfdG7n#f> zVs#kMUevcmx&mr%Lm>t0;X1UyqJW{>&||;>V9#^5>hX9C&Q({>ZDGe|TVs<-u(3a_ zj3Bz@-&us=JFHY4RcO7~)kST*di!hmB;h7w35DzL78`dTEsyh-C~4^v844H~+#F9p ziRYpycQ1I^(ilTL!gvLF9;WoSRc+S^MxRu?t5dUQ<8>o(dXnm8IMm0Fx9f;%pqwge zN`keHCvx@WAp9r3tVFB^n7siH_r;Lk=hP;TS;Gb0329t1j$b}x^sy-P<5yTPwVC&eb~O`S2&)Eg%>u$$YGlU_qpShNHy)F>~tm?xPc1> z>f8Q5|Ao|SF&+dCS+)fn@joRZxas;_wbK$~in-f3( zMSvp=hR{#g4?pBf`Ea7=?vM8%^hC0LY8>aEC(NzA-=*7qq{RcMC}nx*uQvf5Ji9QId0f@t>QiWdC5J#>Qhzq^~U1&F3}itoL2MK($RP5z1P6ni`^-7@}_ch zOxH@Uap|G9x~ufJc*07qVV$s)=OYy!w;7dKx`9dh&$P{*4Bl&^fZ-qhk^Gt$Q&iU_ z?Bb$vg=W{+VATe8Y_p0lLOJs^C4DDn&#CK5uMtiwGCC9EpJx2*qE~pJzVU@hXYX4VL*CS?g0@;@1(ZBum$>fOjn%RvBK*2@I#0OMoji_-{UUz zzU?V1gyJ#6snfVmGhUMY8U7mSTYqp*GhK-aLmu%@@|QX$s6E})q4-&feo5L^^`3{r zUl+_@;mc}yVca!3_)8zJv#(Qg;(qC`@zacvs-fw^pOn<%Q+|x>m~Ym-9Xl+=FnQ9UR|A3!FtaRoud#S6PT^54bEjMU}vX?#$A}~TL=1Ao2I$ISalynqZ$K#q) zX=ilAiS+$hUcnAVS&;1B^1+E`LHOh+dz}B|$i$PyXuki*bu9K4Q^m*PVbzI_Ml(VG zo9*K|LVIQW0+PQ2^RWh&T3N+@ld7}hDMu_hIsEBK>RhhXpNN}Fg(A_y9`a~h=^IQ0 z36vfqyEfAHvq=OyVhISms%}^H8yx0A!!g5<0d-iioHQ)L+Y0S^LJ9^WjE&w`y^$5_ z#>kQMyB6*{b8iXpIzO6s2P)4_TYGo{IO8ea=IHlW{vJ=D~~-G7M} zV?40*GBTI~ruqG>Eu9)NUm)PVQd(;~uy||M&w`tc@7NjEQ%I@V1MKOj`uT-aQL*GX zyL(5`AMu7PqkX|9%}_5do&76Z36MwW{Ni>iHf(?#_Z5&hx8*EGOh2j*a2E0=bTU~0 zKye^$JvP74*ba{e;~%{Q1d;ei61K9F%{+rilOfzfV94ZLk<2^qx4{HQ7W5-fQvnku zi_}P@fwE>t;p@yH4O7%grL~sp1qT*67{*go^_fdX0;r@c<9?9W+#ae}fD=n)ub}B5 zJ`b07bUsal2VxyrXe_UGEr&V!d+E>&laQQH6t~PrO{4aoOAg%ZAR$&OK=>PjSh$?n zT*L(y8RPE?OWyfwHVk>yx}NUaumj;8;et9g*aXX{GApXmFOY1_d*sa2hpBRNu==SGBmnn*ahgnq;+^GNEod&4IzKkpT2znuWx7 zekEA&MDJTuwqIqy6wQ7c#bFWZLDqyE0n>hg|k^^@+#kI3}j-QAvM zAF>tx$bgNvI5JV+-;LYK2$Cy#f;^Z_Wn0mrXCw+?9nMI;1NS7|h3W%z2fa;)nS5UE zs7Pk=L@3O}aY^F}if!SNx@`+w5A$IN*+dc9O8W0OpKT%f2XdgX)kj*UP``}-Ap#yZzk!gywY?mkuimjrTgg>MYW1@+d(>hGcP<~hYj##2?+%~LCJ zl|wm&Xsrx}V~Fz8LUC(B^z6-hb6>dw)&SUXFF%lI+#u5qvA9X*FbxYG#35oKFue39 zPb5w7KibmyAk?G(9K3f@b(xIPv^FY99cV`eM+q(z2wlYL0|B`zTM97gb z@OHf!{+->%8%PGI5*(HQCDNnTdvU~6iI!n5=^B1qg7Dod=RB&XhNVwp&YBzjj|wi{ zz4_qiiYl8E6T%nCse!WDeOl{v$l-?I0wEiRneA9HMZoUsIqiLO3IviHCYJ~VS(+{L_|K}58Yi)ewH8&Bg(<2PLa=l6e@ zq;6AHx@z4NRjC#BOkU+s+jn>S;f=xDY8JpfM2jnjS33$2$<2-5_UW(Le2_Xk-5X4H zXiZzpC;hgtrR-hRl~f!(Z{YY-q(mKJR9bkLhmk8)wi?aaJ9>ag>xzS|Je445?+u56h>vnH)NpSXVIQ2xtpjml9U*zWMMo0VkdB=IU28k2 zPeB|=gbhgmmIHorw$qZPqoOlH$*&ygl?KUVQ?PTIrnc8jlFm#CAW-m@@hpG#LFSH* za9jdi20~9oh=>&!;wV16#{j?kW>JnW7UEaz31n(UQ!`0UMXa25v#!i!$9`9COFb{F z_-f#7_72+@K+p-^Zlqr4wkP6z??VuKl_G$qN`Z2dQw?jd(w{rQtqCIK6>N~{hdc>^ zjZfFl@A%8+Lj#`!F{sSfd2X^slbX>LCO9NPgE$M8lmgd)=x2sczcRn!&v42qHp=M;XPl zQwR%YXrmpgr&=)i$*w^94o*pbiVs46sR2$O29GKbW=9=8S& zxZPE=M5juf646Qb^_jc@O1N#xY>1tHC+kk>amF#X)^mz}Ys)yXg^H?UyZkqml@rf7 z5`d0K{4fUKcljzlj#LwuB&s(5r|B81iCbR;{Kw6b{PWU9IwKQz@^suPIi+M#CGOEW zJA$RzDMfcfX4KDE zj4!3-FQHgt0;>;3 zjx*Po>&y+NLRG1%nx8k!+5hZp#%2f_Wv7z|!FR@+{z$CvwIBwGb@?E$?%Thl?T8}| z6;)z^SwNj~FQ?MD?)*U`nfZ&-~qC9~-cHJ^@^E-0TYX0`;bgsBi~04@PR6NUmyzeU>Z5huPMfS4|N58X=d zrG3N)_QjA2Hlf6lr(Ln9liL||KQ!NOlb<(R0u?ZwawJ24k~t&S?jK&Te;^g$-mvF9 zfD4gGzd+ZpZG;g*<7-0P#7Qs#3jlUNiNEO9P0rP5TlC?=lYZRD*F%4NYK>Q7o$ZO< zaW{+58CTx*#ns_qYF&Q(YiKz17yGA^hWJ@K*^p44_ ze?g&Pl`9%P4}L(OW5hG#mBTM{()JY+!J$x8(X~f7lGfuRPNdLSG!tq|GEgJ$OLwYb zI%+x{Jm}~8^P!2~_Mdfp6U`Du5R1KqvR!vDxaFrf^4zehv!H(|o|uaOcP3&q;)?02 z>3jsHlZ}q9dy$LyK`Xh1Kz}%rcsEV3l^RY#O#!n{>#Kb~PMLy)y4FWv(KSDz8M?xW zDIL<#j+_?UBssZucP-U_KHEsd{$(U>G{7RFjcV1kg>ghlLVl-Mv3IXkM-(>}0nT$g zJ;n~PScCPMW=jgTzFh2@TT(fqo2S+Gqum70Q_jojpQ`nBlfNFatug2+E zphp%|m*M{cHHxMw*Wsi~gVu`vz*s#4kHLO?%$*NQQexZ@0hqo}Z?bp1hRR7!GUD?6 z>e(KMOlOTqT~}t-wS|y{i4bgODc%OQQaL9clSqa+P&{`|QFmDwQx?c<4dg&EY@^z&DED;BW#;<(f)EgF=x4#rMo$JSws)6p zD(@dPk0m2=_)2B7yr~77PWS6^@;1xv6x2%5;3rISlY;=2`U)BYpO)?JV(?hG0%Qp_ znNo>7R$!1%O7xM})ZX3WRj7DY=%zmK!bTChY2y3vAY6%_&PuRgg7%r@>e?gv(Ymwy zw19f)ck4)TAaQ7u*b6jr7hjg3!^3a5k3hnuz-H4m9nz&$R$J7r&vZoLWfcj42QBD1 z_c{y%!%2h>LISW917G$9HLo;sNCc^ZfCII1qPVYt zW#TNzcRq8gKDa;CiH$Yp<8QQc598Rv6J%RAoeV8~+y)CwYQfRe;Bx^~tl0Xx(I2m@ zXka>S7D$dWO>6lyyq;99Zx#6^xrfy!&ZHIw;`fjMEE3NO_wi-yH7F^(^%>41Vu0zM zUYJX@&`)wM0pc`0djPpAJb+ZfjZ|B*6CXgrfVR^^J_hZuU)68pBM6M2lCHeiW=k0; z@oKMn{MgNS9Q@|a@QbH4uov#9B6COrd*H)=0E)Ke3D|IH@8BK2RbI#mNcP-A#j)k}@tau z(Y0$P{A8Daz^+KSR|@vRv(#8BQk*q=kBxsJC-otbeo}CyuF6q4rji%pG$$6A?}|q; z$y`dmgQ)!~NC%G$(f3;hmyq2i1slM;`>LE*N#TN|ad*fzwmY&%gl;$utEu4v#}bmx zKd2Gf8;ah0>j>SQTyDg*pT&T?Z)(#LiveeAClbJb}~SFq5B+ya6_H%g57q0p)(h$rmg4R0})r%?W$^Hjj_&f$T-1FGPjvKxG*a@ z%Ps<(2Uz_?Wi>ebz1r~nsEM~jL77-s)7;ul=V#l3@P&Le4o^H$Z+TJ`kBjO-OqP-P z1;@1itR>k&7(vLHi5&<$Voham9+3!wdLwtrd2WGHc!|1XMQ#oZw+<{y(j!9zUvTmC^_#`h?gyVkrpmo6sMLbG6nQS%3SmS{PP!uj!3d( zBy{Z?J{mbW=!nMY`eFsAHpU87El7zf%IEtzwuRn)`qJOx{NQU~*{t^dJd@5Q7&ck! z3__Kl&%rWmXETor817$O*x%v|ciX4VUgI;cB7-YseHNpT7Aq`232-&UW!`w?5SU>rHxpDyC z4P^FVJqzoh3Fm&Y*In9PGJ>>K`G4}Qi)b??w>b&Dj3ppQm`&mxDH*bMtbHBj%Bfq! z_J(aWZZi%A&IY7DxP_ifcf_J!{ux-2tV&V+DRcVWKDKS6gF#Dcxle2*bLj6k-soh> zTvJgAd2Gegq!~gkJ4VKNdjD^$uN@D^sc@W(hmbX(Yu~X}Opao?Pe6jm&xSnqiw?Y1 zaWg6T96sKy1QFgy^<4i#Kf)YT-K1_&6W*MTsid!7Zwf$P$8hg~u&4TGdK}vXHEZamS4cXc#96eToh%$o@tshW(wuYMC$$ypDSY&!D>>k6Rq>FbkS|MN_ ztE+LNs-xNVBky|&=RO0=GUVKa6d5QN-n9knD>=EzK!SR*MTDxN#W%Zi%(%C)7;u+^ zt_0cr1RxL41ou&6Db64Qz-k#&%OwF<;46AE%kdr>xf(J_KJ-} z5*+xY>!Mz=xe&dYnnu+Zb$?yZPEmH-g!JqE>7j&pxsO{|uw&y8wt4v*Uqhqoi>@J^ z1peDTf7pu)2k;E|$k))=lDrW-1D5+5s^8jPk4-z{d=35kM%z}7m;pY`K#gT@`zS(N zBMzAF>mF(zX&!6vE68=K;5J@|>xfanaLL?a-eulx-eYbxbL^oOKkIAf0zBgDV8x}t zBv#R|PG+mBe;u;7W%Ll?Q18Q}Vb@3X$98>0OtnPvfe1WInbQFmcI2)iD7mp-hdEN; z^w>rHi#$Z=`BI_Q%x;FEd^Tc5xrs5aje8lq2jERrtZu)0ZZ|@9yk5iSKrMNEH+m2N~rh2aJUbKldv6Sesd33Ki0z$4{I3+{rw*vsUNF*7^zP)^CWEJP*o7_ z(X5i|ql#dCTUI*|tWSxnv6Z^kyy;uS(uf7-5|bFBCdT>30}Hl{AuB$2MiU_kB9r~j zX6oWzh`)2SD?2_Mx{KMZK$otE>zug|;{rmM7LTgny{+2QwsH%`q{RaBt_ZP8Z5I6L z7(}!w279K+VcjJzSy8~y7DyuLZo?6(Ws7qSIzy}lF}F3d&I&$k$%^2oWyuab$`&F3 zb2PFjqD$4S;y__%A>|x&{u!FW(Gp%VQ?hafrW){uAMB~eusKdGpM#CWDAzjoX-p}z z&JP?59o>5Qra{oJkyl#1>YxUT6_w+2KL2qr~t01T^3b7I`nS`dSk&0hf}>S^;gTT6y4__4na7S>w?#|%j!oqfJf zQ`5gMyC z{df$Q2B+O2r_|)tYV&^CAx!0D!|B&ITj^ z?RbUbq{LRHn*uGAu)*J>b!f}AIsc#k*9#yJ|92i{#SDa$9c`vj-2Z@oQL01zzxh|O zxS?|wcG0p}3u4NcIV&Pe*_YM4&4jCQAA(r#G zF9L+Py4F!czs-pyL=qH=`CtzmqFPe0mRshzKXWI3tyE|^ZN5JFc znRX&`=2*F+x>k3!CWfe$CxF0RnrxW_Skl~zl@k*Xt4GV$mg5dQ-+`1o;P1rE!=5p! z=St(x@I}Ia_E=;`#K8y-+=NA{v25r2=#731;!iU3glAs}vmN9K9FC#_;7Qf1?ok6F z_O%p*cr%165#rH3v0JzI0YP3M=po3_pgug+RX`KW3SUk_9om^Q)kNuPPIaTrIhxd& z&QL*@{~EV9&|@Bs$DJ-TIy#&IBnl+JuD#@U+3$+qvA}J{?Z(-4l+#d1@tH6JEy^Gd z_>?)if=@v^vXgdgw^e$Hz(+caf>x*|S7EKDPNT@K9EM-OI+#G9WTe&eG-jswtsPri z{>ALt#@s=1hxq9_n2(w6ZUH`7JQQNWmA8qc`e+AXRua|dVPz|-#TOvcE4WGvd8MpJmaWw z8u0s102YtbzIcjHEZl#n`-eo-UNbK}nzKPyD;xAU&sy$NWsAx9j;$X-oZXWBv!GuR z9z=p0)()p4a6^}-|Zw@|=|t);5+&Ll!8APi`aM|ubqsg37VRpqEU9pV#En{!ot=*sjn4N*^CGdJQ4 zr00gGygL1w#*1Zc^O577->7K4_78Y|u9>@51hzg1FLe~As_+*rRa9wVcY){6%1>yC zC?cmZV7%6pg;k?JcP-DmNDq-bakqbTWZ+VS0E~Mga#IAxK71!V1*wUI$k%2@=}P|f z*U-0xcmail1{vTxlN7~gI|u=;FrXdbJQy3N73yFa{F3aV$epS-(Py^x1`1K`Nlc1L zXiqDiyaMCg`7~sA?lDhhJFr9Br(p3cNMNqoi2lKjPVIEVnLI{-w3Ty-?)Vv>hvFlz z;@$@fhN%2tPm@!&Ey_9yg5XmahY@G1z=C^b-=DETQycarGIOgygtwC_|A7nULg(oe zieR6t0a2H!E57dy+<>XnSheGQlEWbX78|*{!@BSQhYu*lzZ~)_$wsxYFI>r{&#BEMo625h#uB%;3M_C2 z_@1HSIWp(^Ni+7IOumBGkN?~sNiipt9OUsw40HdKcm59wpZyJyl`r>+kYMtD)w(?| z5-Ww3Vr+i#rD~adIVyz>iDjBcS~3Hfp#L8Mv;T3%4=;gQB86T@>*~ z-UAVQh>x#e%ix#14Vs7q-RJMV^P@9kDZe2?z#Mnf`~G%EJ#nsf=TdV#DfaZ1e|C<* zucL|hK8P6d`DlAj7f{daeGe%d%ICVTmHE&3J~c5#=J>#L;=TPZ$27z=Tmzy!&@x|d zFxhHX^LF?-dcJGso+x=;>6F#=4-VsV+}z6#Z|@y1CJ&YGgD6i!ONho4d`ufXn)Epl z{onH${n-Nn@AmJmrU(?#Y7q5QGZ$Ft%D*~NcmZP7)E@kR_Ar8q{Yv~ecGMAQt?x|4 zhoEr4MyQP=ny1&?)I*=@ z{zgkmBT@1X0qSA5G29_oMfGGk7Z9Q_LjNLNXkA?veoh)hl9Z`2ZI_&py6BSpGgu;qb?rKLC0gRT-#i(5OcV7M>0M78jkn+6m*E!h zG7x$$FxMjJMJvl-#Y#Qi^w@zJL*As(}x(k3_0B5wVgxT0{-Yq@vwD=^N z?1LY`QW#E%PDzhdmIv+EFx7e`7Y_o&G$(gz^WYBStaTXiF>r-1lo==}qGy^ivj_%a z3^3mnGjeoRRay(t<&5`gx{3KXVqr%KdGod^?OX*q*M8pz2!+|ZEVF08gcHiOAcpM7 zDpev1mfW*f0}@RtQ&gq+&Z9K48$9aPY4b=aiqDK!RWpN4tRQ2H6z>zF_t;O16%3H#?>PMmPIn(rwib_Y|%p5{||4{MTpNx5F(T5+PL9 z>;eS9o=|@y}K@BXSoXIeN zXavs&6EO|22>MOi9k_=QuVg!O2rjYjMCru_oS&Rv$kbzCF>XUrI)au5D1zmNX_ZA2?TgvaPy z39~h*wr*UHi-$7aI*d&-vnLG@Nprl%8h=ca&E~LVZA~l_660|Y&oyCTiUO#Cb;z;V zE#Na>>!llmG{4iS!{J1JW7yMhobqlA@VL9mUuuxs81mIUu**b5+eN_Nn-$-)pjV&U zOHG~X`dCm&kkj0{ajro_Z12A4d+g*8<-lhBU&vEACGxakKsTWASn4z9e&e)ZdmMRE z+}N)!k55`KfMmH?Rm3Bc@BO|EZS(a$HO|-G1L0}3^7{a9;Va6q9nlKpH)`ezbtdBh zR%}}6T4^#yKIn1RIMuciYXHu^5`d*DSdI{U2tJ{P(~yXW*_HT=>O>(@H>mF3juM|h zgvGK;2u~hfrMkZ98bi$W4?v6;O(3wi7Uh(?_OLBm5zF4;!#KRcL2Z7V{VC2Gghrce z1q`0v?C?jSFdCQ+=>;}1?48jcO%^*BqNwj1$vI&jdW@U)fm{?Yo{1!74A|7O|HW6p z73g}t$wO?>0R6&-I89%=R{4`)0RezLS!i(X!}iN=($(28F^skhe!nrrVL~~8Vty!0 zE8G8V_wM@Rq#Y?tj&k7wr;W$j@5)r>?sa#P zD+zT~w315ThDn~Y4MQ)gFs&>^$c97+@li)V+in%303S!e~NO8eTZXok2oMZP5 zSorOzIQ^w70WGzVYLclA6`zW|8{Fj(F~R3WXmiDZ7$W0dJs)Obfu-J-2}o8zsJP@w zY;}oS+PYK%ktN>_DXHJS+Rz<~uf7222LLgB#cZHuzRqb(+qd{qybLuybY>r_pPYTX zlITz^2Vu#KvZYoo{OSuU@+UCMiU{R}w_n(_oY+g`t)*_#qun=YVU0jK={lK1y|-2n z#hy9XhPfNkif}axDC>efU}kD+jRY8a4(WY~8ROcyj&_D;G0!@7{3KhGk$S~|PK1kO z>lfLone8Q7VlR+6)iWuP`-=xq&wur-q%*?$%E1KgQ`Lpo&)>bn1*RHg4>Dcwb65_e z3FdydtO=g?(z%-2#6bw)_T^e7zcmEFh0fX<~L__9-ehmF6Rh|)t&r|V zq7W;&2-4Y=X$j1n!oPuYorb!l7`QZOa$0%3yuSvPy`o-IZzNI;N;A<04K3KNz>VhU z4cZ+s#%O!enbIwkpPPz#A3-PAzGI>6AyWyUekfVwP zex%&0d(drlf7gXTBF;ujbKv8nE9^PEKao}Y$kN&JLe+(HJC+IXLUU9pi@@6h?Rui2 zJ{Ke~WxZpYlQ6$gP=abWs%`0VnlLV3Vcfx|G~eoE@OgX%2$vJPMHy`-3>;2EO0e$p zBAWhaexyLhtNUU$MESQJg zRY*HCP@2qBt~HvhYtru2Hs~63qMb~$ zbahEeg^mn|G6{1Awe?1?AHlE{oifEuusP&kcNA08d1$71aL6(hseo6fQm_hqm@AOg zL?-D3ldcr9d>v0To8o}EPwgR^V%xx6UI%kA_m`6NqnXkbqX$b@5UX|_SaeY{FHt2s zTo02V7FF_O&>ll|1^cF8W55|AZe6FlgE}lVaH+Z+4YXS z-t~IN-gQ`?-RH6W*1K=_{r22$G9rm2gpgE3A%Z{|MG6oIAt68l3FRURgaXPa@;lv4 z-+QaNs?q#2XU_kP1Ce1SIB>ri@+Ze84WyyXL&BgTUuan4T^bjiGyOW;)A zEB^=7gQ|nrpCinX1c)ntj|OT6T-|9ydbcL!k=Yql)1LkSA~vo95yQ^so1reIW%Ua^ z=$8tlwj@>IfStzZe%F~L&GYc}Rc4q437?>7VBBjJ(_S{%QrXlr{xd|lbxoyQzAQsP znw2Kf1NKB(%ovW8P0oPv7MPCdU;*P`lO8fmA!n6T}w=+70Mjm~#BLZFYU&{6*&FW1L4@MHBpASJ?qxGu(xoR85QKeL_Hcy+c zEzlNfH)@Nto3zDR?PjgEM62DRWmv~Mr#sU*LTWM`FkGlRqV8&>&OILw#@jarnB7xq zvQIauE>F5|B4#EE*LA4RHayo65Rnj|Jr3r?btM(V3J{iW^pZAig}!hr^u=pk_Bnp5 z|K2&|jKcrWaVB20dekLSyUfMF?|Hb#RW)1yU&956FheryV7vDDrO|hY=nw()H+;R= zmBj#m;x;(QHGCoDqO3Nd9MEOR$Hx#SZE&T zv*WdfQxDqQ;f`0o79}G6*x!vlsu|IV%HJG68Gz2g0d_i_)1mOCC?4BWtvY<@MyzPg z?;bEe5=n(5fnu(O79|WysFlKHkN6})0(dbnS23;z);VLp5slTM51-6yy$C3<`}nm& zF_P(ZUUE!gMYj!pLrCF$=c^4)m(cc0Xllx<-C;~#fj%*?=U~R$M0{KhLS=Rsx;qD> zU=HezI16J>$38ZZqxnRRa7l;*0VK|zxU>TzfQ0`uC&Z6%Q&HlmtRo*f$1;ICr;>IjN5rqGHW-F2yUg=Gb zmTA)tBb;!Vl3Ri3Jw1s<~3> zu@IR4@W-o+SXQn6@9*!L3d9$!0O4|wFI*iIKtp3Lk2Q*xf*=>)tI;F*VamdByjZNA(b4nzs;`@;nkXd*Gp&EePNBJNXK!l{#Bi`6dd3(aNdSOKOV0VPQZV9)vO2yX-)iiBz&w^X9! z{g5B8Xni>uiEFNSFM>o;K#-zafK(l5k=)|GP7mb0j3Ux+*XR!iRCESiJ4!t{cVLC+ zgqGy==qoNBd$tYdC;G4ZWH&(LOgd6={@iIkB=RoowQ(xA7ycGK)we+O3EjLbC^lG5n;R`o!at%GUna5qnW-Z|w{liO$}X7F(!(A^Qe3L5FXEDmjtR$U#QFp@|w&-gmJ zB(a+(rX5b3Pe}vZdEtS;d#&3)BI>C{vfgh~#%-NBgiET2E9LYio$V4d3ya^Rx(4E2w+6+dn6Pt-+Urc8hRd_*bUXel(NIg%BhyYa8t?nP==E zp_(HkkYj5wb*h zjX$Ag^X{sMzQiJ6x+I+!va`ECIecgME1ZhD{2=zLW+YIk^#TnGc@54X76Csj-HL)+ zN$o#Q6blPU1s8eeI$@utxNG`*IQ%N~n=(>VmG**lqJGJgBB1^zn7vE1^Jwq>Rr!>C z2z&T{WnMW?LgGo=b?F{R+}&~h@TYSr2`m9`@dkn_fIl9z3xLEy7pgt$BdP8fA#rvp zsNGjd;w<$q4j;=1z)@@k9s!kF<^JHopAs7OMvDN*A{rF4$^U_np~u@!Nh9ml!n5}W z|AKfWQBs~oA~lenN)uPCaqFy@lmSuWbpDyKWdzlDDF|{@J8~DZ+Xkoy*Pj0nr&>#t z64^F&Re5e0(GPwf#I(O2H5eSa#g^Be*h0h#Ib+*G6@0ZnL_|9>o!6fh4={C==4oU? zCQ$N5yEChL4jcy<^tx(ZZ!;!2G<5>w(-f5Fm)ErkIFtFAGOT!m=IVo24AJ_s0?@4D zRdd*W2L2>3y_LmVe3=(C4O=8o9krzfe4=;-nX&nB2~2{3JWu3xGoS2PJpE931D|U# zz<-O3>nw926lUWa>POy{gD@~;tYCU@U>?Z&A*-d+BVzVd`N|;Rc$fX0MBHq^+ zO7FZubyt*`@G<7plDMgP@BLdE(e_cyD+{4HH4C9~K6g^t>9dT4zh4A{VFtvav~Pl_ zWAr=kR(ZuOcU_>yTdE_-*C5AqdQNY6bv20Co*sd(%5teQL5m|}1OREiI5kU$8ki0> zc%>`=#$oe>FG&rMwxM1tt;Y+h?e}@Jm1fbAs93`?H4C@j?*EjCt_y+5{eAG_tA+1X z=LiABXb_zb_x$#1JkN&>|29^*CrZ@exNLPl>-_Vd@;z`Z>Ts+C)*6d@5y-xHPdQD6 z47z=F!>uk8?!d@%T&LOlm4m@VUvL5NIc#z>K%JLt(A>ZJY%PKH^SmTq)tgjoB(N`U z0IQ1*a!lK-<+Uy%3~5oK{nv!NDDu$Mcbf4H!3nsPo=FuJ5;%SjeT>a)^TY&+41M!8 z_{GeH=(FPupNQz*-i^?EHc>(XPk1ZH6aR#NIp4KA68|EvOYNFZ;_$NL@A}NHb6ipF;96_P)ayJPuuV)4)J4EALGJ=K5Ph<%HTt3i4$CI41Iu0QJ!#sa`0y^(=u+*dvRJ9$zq3($ z_wLnT(XX46fa&meOThHgRnI%)y-UE1>rWKkW^j3{OmD1vK-sS&G8)vFzwxyqCA)Pp zDD{%yetnIUBUtB$;Xs@XwL&g}dvKIzrCo4>60%Wk&(V%Nw?44huTO%b1*I^N%1Pu+ zBv()#`+df^uWY*xqx0s6Gc>@T3+wqHviIdd|6wYBJe%JWLN+SqUd~CQ`{kTdRplRg zk9avU@n_tSo6`GJVkRcHghS4!K+Ka{V_u?6Ce{9asQMRp1;;p_%7|Lp7Slb7&Pn_d z1AUY_m~25cl~Ub;(q2YhR^1M1h-qi*g3F)-s~ z8}~{Y|DDglAYGN?pu9@@(Z8@&Q}C$me+Zha-!9E=jii;jX$w`>h4ARSlWH;=C}&Rx z8sHIXItlF|llcP>KkixBDe69ThxDRjfO9%MpYk8@JMUKss=x(5Gn>u>E(sw%*Ex1p zSGX8kmOkaj^$MZnS>mnZ$yEL{Z^JNAJ}m^kM*@xpJOBjVZ%ePr{*X7~M(hd@_LTPU zs==xah!xT@mDbL=K>m^l8_Wogg?x0YjFN+%5MS z!|fwO@8Hc44+2m7$qtc0f-QmKTljCGcO5I+x&lXEXj{v&uT!D{`&sot)j74GCP4-* z0};KcJO)L|gB+t?q(L7d=E@F0HpceuJ%+$7pPS?4?Onlm1yxftUO^FIEF3T$A?}XJ zz5(^bdQOAe_X~mPZ4#a8qBZj2To$4=I>lIS21d0th2M28RaXRyOnFVJwi-Gx?^}03Jm}evzAl2}% za?&{qirh;N);>Y~o$i|{>{fnaBQmUx)>IdxF8*%^{qfonP=M#R;0Op<(s zl^|SB$prthkzD~Z(?T?!F31>JOWHK~Pz`ru{10so2kiM^#3`?aOJF{h@1GuI3BMA4&w1N3Md zdFQI%V<-DNwRlp!P-a6U2fun`UqVn>6$Ps=9H9zJ65V4Q`2E-OxbD*4$*u_{e%dk+ zTuCcoQFC=ycjQMA2mMkIbX`Ma{3NenLv1Nlx1Pf?YD<(zz&i7}y<73Bituu!{>A+x z7-{>y{Jy(OrA%aENSIX-ig?W@p@yeK>D9QugdUN7CWV@r6#fQf(N-}(?Kh>p>(=B{ zN4byNco0|;COdibls1h4{tx8{Z2OtrH->23;fe=A&YdfV%xa}h)!+%%7hc#Km-Qp- zcj$0p5D5%rL1a0w7k)aJ$A!+5g!rW#Jx3F9{B_`NrBKiB;!Qa3b!3RHlf;9J^BHGH z#^5H$dQ8zOp_2~6Q7R{e4@Xx!)9H<-B9q(F!EP=mtiGjqGB0hHr}1Sa*=CHSiwD8C z)Ek%ws$nn!M!^`UCGHEMIuS|2GUhix7#VMJnLMV&LS*dQ_Nbc@1T5Db1{lHW4%;=k zs)zVM$?_SXu^CDJ~KE5v@zl+1BV^9vCaI;xXF-VC*(MH+ODcTHpeKh#C>7px!3<^4kOg zfF29{YqT;lfcM9!;r=7<;?ia*2q{!6(U^2L2BxmV$w)j1@qk+4i)+6S5Qp$fg;wN< z<^IW^8&g=Y8>vH5wR?shicj8iSY~@297; z5b;iMwg-Rxn|`(fUD1=O>MRi4b<(tBCkD8>Zs&bn7!-H&pWP3dI!3D~QTNRkfu`Ky z;mE)n6$uEk6)c~L%}`nSm}Tc`v{%xu`kcJ3pQL`4&#-Cmkr<~a zu+OCV?{367veTnMR^u@j1`Ej$Te5`#X28F}$EBAdm0S0kf1K1L@7|*$f}U42^zJUI zt-2}%kGeyEc4$tKPs$Ek4#Kel%MkD=A0<G6s%J!$8|azoy4uf>sdw#Rg$bU5BM}@Bi{q(h^kWcp2#*(@)vty z`~-(mtk2CWK=>BMo5^5`?eDlT{v-4evMrx0zIFCm7N`F6dEQlMclA}SCW1#S2yj2? z>Y#Q)NLM#VrBtmWCkisNR*(L^zgq^_5wP80PbM|`iM=BP&wS*x2By@#H!j5x-z>^w zB)d&JsVX9>Ijx5F1&FPN z`Y!9Z21H&KoKgX==~%!%YF8I+;Iw!KwggzjuhD&6q-*=099}@E9W4gtt6?+7n-ciR zWKGy$3R?Lq&2hkTM|I!laqyGinP7qkp-?_GZqW^wt3Z@{L|dVlJi1aJa+maErLHPM(GBst+K&LpfT2fD3XUQn|FISIZ5<> z3wwO(wpX|z%}&RxKvz8RWcJ+G?l@XXRNZs(xEtk=Vs@%Kx11vQP%Gv!Zew-_Y~*by zTSC@9B$I?7KA94B2}EP!ZLs(deV8V#E#>vTbq{GiDT=HIWTl}bSG@%mR{Oq&hSLW4 zX^~xE2!N`4Ux+W%F6Nq|@PQBpm|+AA+Iapmp%00+Iwc?Z5Ye||GvG?z261w>5SR{H zD)r(z#STF=qkh~+`JFevd3DdO7g$^G z|2eWLS;{6e9y9xLCx{f z(U(?2U}2dz?92#N(>g*9j^vTs%!b9KyJy!C=uoBZC9~f=+uIjLK%Qb1P~*(u=!UBN z2)wvrm2TPj?UL=q%kabZ@}R!4_2~$*)q|nlF8yZn=@}hyd?>#AGb>M?&O!ZvwaUES zGwD?G^`pyi-0`xKf1W%&iuv<#-Y)&cxwLKgoLk^+J%7Br=N_hx{gby#*Cph?j1>{j zO|u56!1a^Es+J+>gZoWN6aXgu*Rx6Kza|!vmyiCmZHn+B`li=!nZL0f{dRt}w@bfO z`l!o-zQ7W%kLl23?9~KK-f0NCsm(Y0$g-Ckh&rn9g+whFO_r2b;lDwnn6~57p*yz>Zq67&iQ1+Q#ch=B*e9oT_nr3 zru4V;sIMiR@5`eAQlO4c;8i3)|A|^Exx${_V-cG95WETGIC&YLC%;5h4+M_&JZ42( z4^Q4(c2Gh;n(!z{5KU6HirSCShF>Uk4o=8(@o-yt6& z5{3b0+?o}-Jvcek%r@{VgpEQF`i8D8f0B|L2xb4jq!(B?o^hP|?y5qv^+2vImx3qZ zcZgpu!$s8n;$j6MRO{#XY6vF6Z2qsyoyiv5T2kmk-lZ;2ZPgP!n72UGRn0u~bA3*E zC)Mis4_<^GYFtHrDX`X`pnc0SW%m3g_x`OA9A!Ktdc@fFVeQr;8%+Y0lzHdZnbiFo z5frk3@>b-O(htw2o5dqyFFMMVF=LxM4LCK30cMio;$Cpckb=1w3@C3Fz!Xvkwp3#p zzz1`nTXGd4uoPWf*+CQKO)T(F36{as71ti55;L9ieblV>WJccrV@b(>MNrOTfO)fC zgj;ZS6&mZ*FDh?;fa@MFYj-;=p{5ggY|IhF+^{l^^yA`X|Dvr(RQ@YRX?D)gZ*cKw z`0;3>U?3UtGJ~GO3#z(meDEEbY!rjsshYEes=l6xbq}zJb z@~QN;+{O3^iUpQi*5PoM!1u*)%*d7sq06)=Gsc*mqt&0{Q+|mXa@(M?YX;Q#%2TD( zncXi|;G@jhFF&L2(YyQQDaJ~nMJ0l$$|&G6r~fsxwrnYmEpqt14L?b!zDp1gWKV-s zdf?evbYSzWV^_Q1R;!OV%6oV*F7rCR)TQ~&AdL335~C?`9n66=Ua*lsf7|-sDNGM= zg_7C}boPy#U*IRAC~%lQ#n`vCgyMr&vC8k?tExORWX8j-MZmO0$_rtS7O3Dso+S@* zId|b|{h?hAEbdh;^)_C(NH^OOXgn`bf>B?m4(8(q*moVP?p1(2h%6AFoYBsd2>>JO z2(Z1RyIlcusm?^sObm9FSyLCMHTP}~ZzVR`N)R#boye@a|4xo=od})mne}A_Z5e7o zP1g|jX?;s4XL(jZ&$d>OTA8n-ivGMw&8>$7O4=>^Q?65FuIt$TYYq5=qLe)%4$w!A zHTx3>;7VXk(0V1o4cGWhC-S(;d;^4S*Hmj1iA4o4d^&{(?0X_iLT2USdO4sb1)l;G~guvCYGM>!#Ag*ob+M3hi4=J!DCRh^s0l}3~Q=Mf(=;ah_?0q3;y_K_#8`y$cVX3a@hf2M3Z6pv?~G3fpTkvqujs?`#)hQH+)EcKvZ@#U*^?2F{UqaYVq% zqmndoIS6sH-jWO+iWS>t>FsqRf50J3UPrmv=S0`mUlFa6#qtqCXH(m90DeG$zt-W` zwh)-A^-{lSX~%gb2*0jAs5+%Pt?SmE(OptYEmkWaoqgPH#)oYbFrM={>r+jDoo$fH zv3*H^Gi67AJKnE5pR)=dvR*)J$(jp;bqhswV^f#@03#l}!6+VO_Rfi;i`6D!1~HrC z(q8du+(B9lECr!k{5+6uI=GHQknMHe+c+5-`_%?5(G80NhRcDZ>svan60%W;#HFRp z2X|9vC=w?HYHf9z#sdBea%g?SC_gPofa)A9r|bk$OVgfolW?${aMg_mfjzo};G-sw zUkfh7{ghD1FGo>Vq0_VDM^|^vQxQ^tpOD1r@2VU=%Rivu@PiM~7uKETdABI7|wB3oJ}y z-3=?{;>aVbUk*ZLdZ5IlD{?wPz|N0~eM+`+$Y<-d6Myag&Q30*r}blruD`3vNMP!g z084ssBk-hf^uT`VVHOI?!SZq$UDJJNKF@z9-&SbTUfbP*SwBwbV0NhJ%S#=kCBs4S z!Ji5#$U(IzuNnv;gdxEF+CFOf5dVY!g{AJ4XoyqHwXs$ePEi&C6NWXq_w&KPhuMk0 zB4GdW?Ta=|+ud>%Mtgd}5?|0{+&H#e+(1j;ZK*c)&^Rrm zVpR{y{5DvWMj;ayFMDfbJf?ap7mUW_^Nf-ordK{h=^g>}CBYp4OF>Hjxbm(Ur|GnQ3@7BB=Gc>(Pd@#N&%X0r?=Cu^ z_xvGM&NlcX6f_n9;~mv@^8x=Rf43UL2KaG~oo5InJJ=p7__%ooQk*~>2rSe&KK!Jw z>yX3ap`L+X&Lr~~N;%I^rDlYB^|U7*A4(pAGI>z7t65<2vXvEJ_=lIxT#l80F^SQimP4Fw2O_n=s2O_5JLp{saym}&CoA?fH>1+S;L~#^` zp2_n#Emzhz>WM>D_Tb%jpaE%F@o&jelv|kM%@2%Xke_@HoPVH7YHRVwm2WJttT){Q zyY!8gJ^H{%IAm};HD^P>t>>|OMvMYdTgqctDCa@X zX$I3O2*c{d!0e)fozQpcF+b}{4clu+2$hvi(}6HI&ui`?kYU@c?%cqAO7y?Ialv=8 z>HQe+8$4gH@i?b0&=jP3!h%WbNa!gc^TQK`wZrCL=PYy1c`8I z;5vG)&HeKWMTGESU;CvgU=OiOIlsVF3(BANnR( za7E>+u}E~1WCz@;iJV9k^aK@1)*0cp`ek!@98s0T0?T7rTjyIfA66&w*pB#35XI>- zRGBJoV}sd(aN!jfj~fqRK=;Htx9R2d=w$KwOr!bE3>GwvcjUr#y7#-lyxy78p5sDb za(fe(`C&ni&v`opBtJ4C>}v4#K>Xv!YnB%!@R;+e;6pTBWA4ote9-!{0#FfzKd32J zpSl6P=&u}isR-K2Zezv#f|A-5+&%=gaiS+@tleQKzqfsQ`QGp!aF+ZoSnM95d2g13 z%$7__=ny_7>E*KI=~oY_{kO+L$v%}?Jiot9|NI}#O!qem;VH9d&Z1=g#?(>Uas!PAX-R6%+!@$rEii`sG7pIk8oKT2@Gw_OUo`7DlR}Cb+VcW? zKJ#xmvO8HtNNvD%Axcahku_e?vDpU*-b%3Jhsbz|V_Tb#uGis-Qt75r_)%l3fUV?LgNRC%M6X&BBo1^(O^*pnaEl@q_+!Hb zqC(vz1oOLuV6r=IA@g*MJgA^rGNLdZ!aINae241zt|331uY14z%no(W_HqP$(wSF) zu*EU4OXA|UX|psk8$P|7f;AXd=&aT}v+|qT{*D13qPUb-tlu_JiZ#oBrL0(As^0tP z93a{bVv+am#rz|<0zdK?V7}%zC$~$e0(oRA@gVSm>3~X32&o)qQB8@0AIT%wr+H>U zeQBxqn?(64C?7&vrGxbCs_e1m(=T^73$>Y`9<+jtdk8$eqK8QQ$ zf6TYp_X_hGekl@$wbGVn(*Gu*AFnYdP)=-w?~c+}G<#kQlfBY~z*MGo)Cj~?x`oe9 zoTZ2wCQ@o;@yK-QglkBNivqGQhW+FpHYm&aFky zArj~x2IIjt*bTd=>SUzTXi^rsx4oZzJSPx?K%AaZf0CIl42BPk4Tckl1{v&go~q^W zBeNJIMOM}$KBGQks6CntwJ-tVC%YVk;uO$Rr6TRd$HG3ilqkdlLunpbz~XYp@0J=Z)V6#9Yg@j+cUtPr?l11;t{#d=p=CdI z3)S1Zz?Jim+u8bCboQ;`KpbUP0jr!YB=OrC)2OW# z3BEYnmvhRwVV#m-t^Ic_mvgIs zi#LhKNA&H{<@XMS;fi;m3?0(0ewmhR5TAu2)+M_s!^hrXTZGYA9#`x=s5?>Lx&ZgL zJgz9?)b>=HAO5vSatyO%1~ySbDe965DmhuQOhLE9aQXjk_o$m#I z#(GG#96?@gyvCFL`D8+Uuaoof@@v!b^=Ilw-xftD$Ez7&-R~}KT|yrQ<2bDt8;u@V zzuS|>o{2Mm#^2TBlj_rq7uJuwDQa1WMA(?JH(C#2EG#ScScRJuEXDS9j+=nre^qC2 zlWH?BS!=F7?xFBfix6GF9=@nQHhAhEJiaG>D{x9Z+BOt`-@gp+R`>q+g?eneua~{% z_sKW1wZxQ8j(Piny8WO9N%cTK&zJlxbo7Ho-&en^O}eE{NRzzK@{hTXBfh~MCHWhAwuQKt0lrmB2L!-qHpSgL4I%np6) z+4i;sim0VmfH2u}8jVUH`py*8{T@I31;95?+EtN!(+Ck7gw4FFE`Qk`k5^$hV2=0~ z(ygEkIG&~&rec&0x z(4DQ|=nM`6Fd4@SN}-h@FyP+#M2ahD`6prmDmTk!o>2Sk16Wwi;z($o_}`_s)7A72 zx`wWm5X?N9u~ES1wn3CVq{Ahg4QfO(J`Cg`1$d&UNVXYdpJt?4#cB@9pit?cG7RWm zxd$(&_GVGsUHDU!K(7N!(^Tmy7qVr3ppO3N^E;;Q!uobVR8)a7R9WYSPCuDUYZ9FcR2{nqOvLR#2Hr^WiV)Z6KZ!k(G;^*@`lgh z9aU4d@FA(XB3F=Zhd%rv-XB*FaY`jF`=p<0SYH{0w@xhZe<5Sx1J1xh^h6pL>y29> zX`1cg=;@RkC89}Ml=)q-2qQ+lVdv|&UHs)M;jnEnS}4)L z^c+aUq!}Ae38?S3&XK)H7nAP2C#iNl(Pa+M$~rUJJirr6Cc5i-yQX#(KvcbGat9i_JTZ1vgZqcjo?1=<~9N#RAlGKvCn zFjMxxQ7UU=dL%y9t$=lngq-P0JItqhQ~79glf}SKOj&o{;+G)){-MEIBm&?^c-MaoWn|(!iE;)Btj>w0mdL%ne+tT2v&vzaYBCPM_;6C2%TXfPHFGGK`M^`rCR8 zP1OBE*PRLS_?Iy^JFdz)$B{{(q84TnyQSbvAaOUhqk>zDQr3pCQF~{$Erf`qik<0G z|D3S#%3|E!vjQ+k2!1vVn7JgVL%T5w7;F(F{MaLWpq8WrF4=m%i-HH?_X#dd^bx}z zdM~|?_5!nH_YJ7{9G+k`td=9hO8WA-pKavqEKWz)f(4i9EA#vn+tsl0-u{+v<0@g! zdj7BEU9~-re+A7-<|^HLZO=~#uI)0HM3XE;iS0>JC0{N@r@Gd08aNi1G^0C636ouV zF|Jxk^5uc9daK}1ETd4MwMV!J5F;>4)s!KuB9AjKD)UVJT0R8>C{b_B`xE3S3|%OtGWVcbagCQ7rbML82!ze`y!xD5-rut8 zhEY@!1)||lhO;rruu1Jd2>zGS_x4xa3J`KAmG;&)DqD00{avMTl(p^IrpDa(J zNid5XV`ppZtE1%%?isFeIU%3%=7h}LUXW8eq{f{{c_ZXWV?yQj_lZ#U#yom0MZvQP z83uerxmZ;q>oMGP*nEXbyD~(ys1^Z}3%y5HYMlPswvB%l751hEwRA?@v+D4JZZm#2 zD?M>zlH&Y+T=k2=XbiT90cM-bMt?!Kub=cIO1EVo7>^GV4Y3bsU6;28i*UL3i<2l9Dm!U(iq(0 znbV&=u5Op1Wh91{oN8w<9xMWY#0ZQgt+O?1_yYK26t*OKEX~upq!%rbIJ;TqMeLT{ z@|bSQMl&H-1ED?;)0cS&)sXa8S^F9zY5fWN37Ri&DaQt&3YBpu{dDKcP-2C}$ZF`b zwke9sNklUagX87)h091&vD@JP%p?fDWQt2_mWU}j`K-MJQler(me@F{FFrPEB<7+q zz&vJp=u_gm#dj1GS=FvyPEuxtBQVJ~B4|koZ#L$``N2Zh%u_ug78+OASRZap7 zLC-w`faC}~$j*U~f;PyYe~$wJ&MMb~LIIAS*+mKIw%?<2b|ElL8l)4N6c|1Y5thv4 zI8R)v*p_rii^?wo{ayk(_Kpwn) zV46PrB=cuv@@mYG1gnzZ{e!#WiRAjVJh-0FlM@F?+b}nndPJoRrtbUBfYA3HV&v;# zgVs|5rZ@5KCRdKLv*yNBii1%VAH(ILl5Cw9Li)8vl~7x=08rX zb7B*2eHzVgfY1pqK&R9R?SnJbAL6$eqtH{;l^ApIK#GgTMyH zZR8}@E6wgub(}{83SMJgB?GgOWY+apJ{r8iy^qp}B~JMtkqe+RXchjlDWUfQO&nff zfJupP@6DOprio3+w&<(5U!`PR%kt1ER%vj9mwoevAlauhE8knefZk@F`ITImDjsrM<|EcDo{mj^Xl2iup6W=rBLZltUD0~{d@rBuX8_Et93@P%l8 z;~aYjUk6I+4i~;RRE~+vnz_^HBqcqK2Ec6Q03gCotdd#xt^uaW&Z)e1A5&EaXK!K2 z79VpM(AD|R-QegZ_4Vyzi*ay}0}2cp%QsAXPP8Hx0K)(v6lV0p_F)?CvhBk8zzPuV zQo5-!Y&TRJb{JA4Y7MzW17M3f_c}+o6~}=9Q9lT;v;p?&P(d(JI{m%wCnl@+tce zRs?hEWjyrg1T&*PXP@DwPQ27Ab#s%-en;>jC*GdOh=J&-zF{=Y4l%#!h-=<0F7k(; zuDZ7WRna^6fFm&U_kycU?hH%#(ZkDkg7IryD92cz3EF=3nIGPva?7;Yn0#OGdCs4Qf>uTVvtO{-rc>Pl_F7R!R&&zU!NBt;Sp zh8SSpCo`$LOHDl`!~%h1&tN{X`>Nf6`IAHcOfcsc%d>HMKRP}jsL6r2$pGgJVPG|UvG+q_i?&snXS};lhAOn>OM$te;>Ng;J03D#Na67+?u6k2R zR*Fr8+Chwpbn>gbGrm9zOs0!cg(z}HvC9{)@&W2Rb%DwRSs)wCJ-f!O3tAd%Z{Z5$ zU9i|Y(G3y$b07nAAcF(h*C6z7dc|B#xbsu|be8~2vq|pCi+!5X0qpxnd(|dPo-tgR z9n2=?_{vQ;Fn!~tnh+C{m-soW8$5dL?}_fE>^mK%UX10hMOQ_*RH_sx7i?7qHQQbC zFF1b1gTUh|dAt5b)zR$_HN-JAZ*Q!A#Yp?G4nKi75HK9tpl_l*mW#*uBTligakSCb z_FN#8Qe%L5*t{=53P!xs8)m;+`Zm5UUKugzp^dQugvsdF;F~816rUgv^5V;#EiSP_ zB-wy0r|R#2iSMrfpp^QduL4%_4)9p{8Rr9bUmjYCOPl2&^ihb1Uj$04Zq%$gl7LK9 zgS`u+9sTbUyYEKeki2%j_LyO_&)vh>K?zju#&HU*lL!FPBk-)g50+6}z7wSDO5VNn zWza#WZ)t6z1#XC=$PuDP5qJ2{hlbBsMTdyY0IJVts|)(#4*OCB8k<`8A6#JIU3v;Zjm@z2cquTd)Q#oOO zp@_n7`_{?R6$TP_}Y5Pv+d`M8~R(U!>N)DkNEq?=dwEPW@JZnHyweLJSyCZ8@xq-q@!_}zM0w7srGHeo{7G*mff0)V<}$UX zu|*^OF!Kr^Vo(h5cU2)75t$KL5!n$wIT0Ho%uLE}&s5xxz&~vXu4V(3++9rbjX7@fVI4nGuS2>GLb9lFP>z;N9nSaM90{WaRr2DM2joKdN$j6cOip z4XmN6lehqq%hvBR;Dfh)GNmV%&bp?B$iKNr#G zE=w4uC$>{vu=SUCur$}9L^J?gSJcrpIazA48VN?`kuR=BGuu(4A?oZH;6LR5(C3lQ zW1lBJgl|p5fOgu((@{_js%NgA zUrocqSy5qtm8h;u+NmMBp|U5=R%+&{{WEm8_Z)Z2)Oz0SkgDMXcAe(c@@7qkf)5N- zBmJ+p%!}o4G}G~Wo;~F$@x5N@XZ7NlfTuTyVt2G|l+GG0BSLYIuemB z-Phk1_l|NSr23s6n@>hQW!T3Zr4t~{*?El3TnnPBG@|aP>}uV&Cy?KO$W5OspPUW< z`10OVkmz7{14NGnyLDbS!XSPC=Vv5Z@`<<8JPnnW~9-OCQ1(%eklz8c3)8JV=n&rYP3YB3IP zHHdI-cP}S?Zs~XU+|uvhm-7t}w@>w)dVy8vyy5fOrdvN7%xFLDLx^O^_2WFu86$xH zU@-MG@+`{D`QHIYQ4+@5eH}F_Z08ECY zlCzSk0ef0>6BZ z6GN-5>5L{anJD0sYRUD@^F{H00RAzR6XMDI+1sF~twdc|GP(eVArHMcaoB3xOLYz0 zT7VPG<=*tlnJnsV;h3MeBjhP@R`}Sjy>)mko)#bjnFBkh%H-o_+yV{qVSD%WMT+*+{(oG#lRNK*j~ZR|JSLZ)E%4zE3j~Tb2AGSCuR)n#2_m;J-WlP# zf%rxQ68O>_3j&7C6TY63CCvF6su8XuzNT-J_MrBV_OSMdcJ7i6ZF(tgBdjwAGWlxe zAI)sXz<0SM6#9Z4d94^kuaMXFL)Ac0T0DN}%Kbz;Guv0|Lo`O^cllE-_qY{dFP%7f znezowwI?S&LL2I%`5S)haTpDc!)P3(mVscYNFToX>TXqc!G;epk|;(V3&tb_+lqL? zr&&q(o@d4%V4>8tc-@X~(X`xw?hZYtQt}Yn*{g?Fk+>hE&5^)xPb6=X5N<0G#9sv= zipZ}bl^~p79xDlIcsq~PkQM-=t2=~uz`g-vP9m=%QJrN$Wb4RwfJ(}+0en710>fzw zA?MFNulZF@3WC4~e~m55v`36r_WR*6Kol?>2}lW@1n!i;1Gy6oj0c16(#1ZTeGUa} z^eyr=v#(xI+uFB%Q`DT1Va(TPZlvz~gc$0r0uk~}Fgzv6gE2xPI=-XkWxsUSw$P^w zxew5@SOAR0`qQl4Tpj2>EK+%fOXSp~BojFsLvefbDczs2BM$G;7zMQzsdkX80`jeu z*_)rCnQ#szPF%%`q5BUTFW&6LxQ`sWF~kWlPc;HjvzQ^VyAQF!LK7DRiJnLJDSy6BWi3<@SjN5{t z7ix2ylQ(ePj~EcU1Vl!ve6w2i-C}SUcVh+#oUQYgO$#SmYJNvB+LSll&2U@#1iRfw=MAh^d|5|Qouse{w0XT52*)zJEBGD!NVXK266^8@rh7NR0$zKzt?Y{ z-y_2YI+xC)b2N`th~;Tp2@}2)^(bW~%ts<-w;zcRi=w#ftWTJcmM*V{S0IrrxTf+V zu}}9}N{h-aYauYxN`%}x7zpcNAWop7ficS{PS3|-Tn;Q*TbyRx@9D+}_ru+1Tjc0x)s@VH|=Ei*JpiJih zmxP+T>*lfBVXyH)wj;I`_!(OPNH8UuszT4{9q04y8dGB~(ldWeO;MaOJR8tv2?@TV;AP55U&$XKbstcxs<3Ew7rkc|W6B!~n$ptX;=0i=G z-@okZjJYqaR}HS`-0QHB@+SY^sIgs6`~iyRMn|ALqwY~ZWuA%AtxWr0UOsv`3P)W< z?f=pXs_u#55S*TRaTGYEyFYgQ1LDk=IOdD5z87+_>hd~b5a;F2$F;M&i(`ocNInH! z@7vyS$4ul&ky4e5r1Kwe$B`|H6ZS(&K07IWhif;`C3)N$bQFXEaL#8ZC}3Qqv%SCW zMdjq@j&I_YZmac2Q^uR@?)=T9Oszz(<3XaTYe!iCeykpP9Mw>r)M-Oj2~Nchkp>2P zt&8GIFuFx0{fs*``oR*xfb*DWw?i*dumA^ZlPJCtt|sdf@u!?9QT1|YJjls%>}Hs} z2X!QKZkgK}Nv^%m(O>SW8=3N!`hR@>D0sGUbiTSk9C zp`)v%`OW`IhxLEEdqRGwy5cm5W@M-dKN2`)gn`&=a@cQ=Z zH&=R!8Y#_+!SBz-Z6s0>RAcV{o1O#5|Ah!|vYxW7N9o}U6=Aql9SQUsBXF||7;r;HCT!@Yqe3S#@rNwDK~ZUFIMz75`Zq!Br*%zMt=zU<2h zd>CG<3p`yT5A*K}TM&ADou{>N)_Zvqr8<0K%RqG#*y>xx?WM$BOJR96B? z=kHUg8_-D_Y+el_#Up=Qk|Q$uV9;cKq7HA27zHY}7UW*}lnAN{SkW_d!j6X)BHxg) zitVHC?cB-|BL?rj4CC7Ap+q&tdwFFv*49x68`}fOK|l95=1`#B80w7p;1O)1 zXA1cD&?~b7P!xe>(hK3=3U;U`p&}&0VCQ+&wUfEu#syIX0GlH4zjwF%+Sc9jYkac4 z4HjOfoSORHHRX@y>V#UP2q!p-orR=@vC=u9f0Xn{!#`_%h8wW+7Mc+P))A3oD5l4Cbsh97ox?h%s ziJr?*zb*b-{hR&zq*C#g@u0iMvoUkjqgU*pd#7nh%-S`seumWKE0CeMy8Cu&LMM;%bmc3VX>Xjg)8+;|-Vp1Jr&uw$~p zBI>w@$&!Eh?I=pnA}Cz z`9u~!G{aW*{ncfFs5jB|3hZ6~& z=S+OtH@Dxr;ftTn0$|Jz5=Y-h{J#r_g!j-_3I|L{M3>doQNINnReUCVi;$~PkmQys z-w?d6nQ+_$I{(wGp&o)JWd)#$Ig8LuBQ-3a4s`@bL=MS%KjuXB&{85*j{smxM1$`f z$(d^#n7FT3ah!9d3r{{9{TJL{S^>gj zDiHh!c@Nap!$p593JvlNj_IhE4^sa`WaV+n1Eno1Z`>?cgex!)acXJvyb&_?99uB? zz2}j$PJv*;E?~~8CeoyY1B{&|2Z*5z_yg}UNkmzd$hbg3il3QpOH13ujTTq zR^gxHL66vK*mk|rm*_%xnRLHsqJ<))D8tQIt za<|+Hk;8qqobRNXnz-+wce?->8;#<&LN5GnwVh)LTBYn%kG*fqN@^a%xmU2`V;%d) z5+H>;H`#8fPM2{%LvJevn72lF>R%St)>B0pd=$a!32I;!FszgC9MxDXyoqXj*Z6V2 zYDO%h$E5gfpT;f0D02kR+XG3gAMY!q2s$FKP#P&{>Bh;g&^}$_?MhY0wqFstTd@kl z8cccQiNg)3gmAt(`56Vj=H#Y59}^8=#Va@bz0H37*F;cPoO_gfzahQgT^vke73LmH z^Iu%Q(X;k7E=+&#?#^6P-mh_d$ySXV+VSxLYTvQve~%|~O41`t>hg?)+INT@7N_uY z*aKf=+;1tMh^9b{EdPb644Zv2vxH!+Dn83|3!aRn#1kK`z^FN$Nq1$8CsX+MphwiG z+odCKDa|C7l8fjlxro?@)bqA(-1XIof3<+njZl2ASN>R_IJSg+6`1mIu}NEH@6W?fm85%d?m6nbw*!aBgw#> zhiBuG>=Q>;69I6!S3w`$x~Y!MCgR|8=}l+q_gEwr9Va=nEw^P6o=HVX{XlL~uU<84 z2>gzhS^up1(Z!=-gaV3Ugs9waX}??cA>JU8jVD@q%%Cps(iT#_QhAv&!|Ef0HX66K zV}W^son&8WYM5HWwkdtV(bTdHY63I35?I@4Z|n!=nomLvPspUjJApdH;1RE;M^n`O zy)WOx5l~5+hRKMUXeuLQM`T9XJLuVsXVpZQl-L&>pEUQE46ng?`kFb(FZb4zMPinI zyP>qxM7C!Fc4;!?O!az--`r4M`YntUl1S|xI^ok-Irb4w?ZW``SaUY0%h&O*gXjO0 zV;}l4+AOlovIaW;TH2SQq)y2b{>k5sH75IsW|N|P7fS@*tj$!-Z05g9;^#c46N{?v zZ^Et(|J%F_gd`gbJ%(PxZjI+aITxB?8f3{WO8|6<3)}bwu#I0Jf?#nGRVJd@175)U zs?Kcg8zhrajKrQTFec2l#N&Ok6a?MZ?PKJG1i&MBUNxA7dLJh98z8hS1S7hnC8qG? z8#SlPevSJIQNWPxNB)Q+Xn;k~fPcj67i|MDn!Cxz<3%G+z0R@wU4k1@MNoKI>BD}3 zdOnmLkKg-3IUA-yv%N+j-2FYr@L3k@Zv>LgR-!4T54VlY=5K@e zWL1hPRrOjBDWBbk6Uyj>Z8gVl()bZLXTvWD2A9;jdy&W2)$<7gWf$v1ya!?jjhkMt zGrIqaq+1Y$7gmO8RtWX55bE*m7Xj0efRxY{eXAZ<&7F`<=Io2`SN#yg%@2`7Ao&4X zt*UXK@HxqJIRfp;ij3;#a)%m@H> z`{UtYVmjSgfb^bS90Hl{+;v03Xx+Oaa<%LNy<1EtZo_s*TJuT}UK-)qn!<{d!!T9F zWx!PgQ_usFt2N~Je?pw7@yaF>m+6f?JKrNxc`w&;nsc+IKSf81pSNpgbkBNk1`!#6 zV&eHE{`ARAKT+N3>aI$KED!Pi!UfKX8^ZC*2l@?aAoq-lIQ?Y(*QNpnN~QvcR=ffp zna!-Hv+mc%lNeeBZS?V9G!)i<2bbH?z}OfrZk2e*^?PtVOQ!#RVD~6ea1)$CR&)uw zYwwculHu>oT`^6WKL#s$RFhP%>K2u(NztTga+t&P5&9_ohAziT=td>a7_Jif5JvSoNVg=ZNG(bAl_*&ctD2>*OXCe3Z_NjfD#D|WfA0Su-WN1Pe^Q!qLI9kiA!QT+%nwa2wbeu@Yqdd zrFkugZhwt=|LnIzu_0D~i1q5dREAp7S6;hxV`CIP5?nH1 zyH#>BV5ilaU-1M#%ysRETJBn@P1_P+>D9|s70rAkY~~|LWK8gOqf&x@1b)sJ!U*&P zmy5x#(&2PrT+23&P$h@~=95}aF|^h_CTg2Q(Pgx3?%~!K}3qXh>>q(v398G#R@1<<~KmdBmHB&!q|Dt z|JL1#Q9P3}o1=kW3oE^=4zA-6S>9QORp>olWeU{s3A~E@hC5Y`)93;tO4g(vunT3- z18H>Sjs1G!9F7A48(CRU+0nkB?1$|?C+{a?`B1*_W_zEXXnG=Vj$Ijau)H{w7zRa3 zCYZXlTeGK0A(|B+%v}i*(*F3EYajY$OZ(6-aUekSh6Yum{vNph|5NtnVNqpw-uQcO zZTFrkilW%J0tH-9TWwo=OLyDpPIsr1PD~~}lg>;M(=$nCGM%I+zhq|KnfYZ-u?fgh z$Re^9Ac!ooD4XjAWEB(_5EZ3B1;h0gnJd zZ^-_KJz3TM_>>k}g@!6B{`%Y|JV-F}>Z6(qvtA?4?l9nX$;&}KA!#6rDlJB?G8@dp zs(c-X>60kf0Y4^Yfq;@kvr(guXeN}r1q(SpA5W-7u2S4b=5Vjj8>fTAyXXrO!@Fpn zDhV*ii**Y=7T|{7BsG=6KM|+jVuo17bk3~i{4ij9mK<-iDg{Lblk;* zSjt%`e$@jxJC7!*^YOrwa$D9 z>s33CcYR90*8%}sqT8pl!8r>)=8_6N#&h+3u=bYhn~0#_job@P4w;@(MDJP{_pi{? z(Z;Pj{FE4hYq$LV{i{9%_eGe%b;0Xl$C>}(r{FdcfW7O+iFn#Lne;=vgdFU(6GKZ+ zVwGu&eojfCLo=egAHZs;!9lEO`M7=+cdC`J-Vqc6h}9O}o%1HT*5RVU&ul_3^+ezP z8@#ElbFQ-AyO-u7A+`uw5Bzf&@r?`IgN+hXBE;yE6+lZGu-SI3Js|q^10l?n1$OoXHAq=e4EssO|z^e7LRxTC@o8;h>z%Os9(T zZw?a)x(_oMz>I6P?8qHA<24wFYu65dDtic4L%(J)%E;3{a3m@{NTBo$*o9gq48>21 zp{lxykwCzP#k@t%KakxRdHF9mMO+5dDcUoxq^MXN)EXl=DH*X<1|If5;$Pue>3P7@ zcBzs6Pa4yVI3WEPY}zL)mD#>f94-9+9160X`c4Yofd7P!gVn%G0RLy%vr1N!#iY9P zc7%WX(*$*k5;V+Y>Q}&IocPEVGauQf-GA&%C?B6J*1fX4{q8g>zI9g^4_zQ+s+1@# zo`wiAz_HFz=?;C)5TL2?PxR)${+9lC4@H@jgw~5{$n&r-Vtaw=1~8d3m@UI^>kSY( zBoQ<}wd)q&fd+~zMc~v8+Bd**qgR2~8*e&wS5X$9@1g6HC-NmYBNLUwn|rse>_ia0 zxo|OJALuZGL(gOP5`~DBV%_No4KzTagJb)EHt9%*ph)LBX>(v3zcZ4c=az{oS z&4M&{rfN%j|7|y%VhfKt_NP79T2#1gy9#&~Fn6W*r1zzE8lU#~)m6a5#-nbaNLP-l zA6$#;`85GuS_^zExCTMK-qDS)v+Hk(l3FlO4NIFOkKLcRTT8%YeE{|wJ#m1$7Wfp? zV%UdrMRe*N6>&YMT-;q@qkVhXMOVuub-M8d4s#f+dw1RChRaQ4J)a3Ra2Lc<2{vk| zV?(4km1=G>5qqgH4vcM%q6Z9}!FZwt08pyKahjdm?9ZFkAjm8$UIA6D&y~aXT|)(J1dC1A{=<}{dNhFkv;TebkRrnbJ$=s zj=LlNfl<|5RKY=3IeKRSsI;AXV5L66=<39e}i$TjAyin!Z?aO zGPCZp$&WS?wa<0Hw^g0yVx_1$)Su^u6yuGf%W=(X74SMIZIX;Jm0yA;H+5&ai8HoLc`(O(l8*f~NK2#uJ_v=ox z@?1H?YT21asxJu%7#Zn>-H9yF+9qBhJ=Z%f=b!hEGk?YYqZtU$1pi)y5O$Wynrr+n zv3Cky>(#@Nd8n}he`|gP2?BwAj!G78 zsGs61-wdyUkql1ay_qr{Q?App$NGI{Vi0P7gF~;EIWNyY_L3z6hh;!bh*4P<(@dGS zF^h8}NkalbKPes542GQ;%x7ruFtwP2 ze?odAlU|k84oKjSu-=xbEHE|#QEGjh^4ghw^5N%4>|UHbuf9{=oNXj1O@2WBLQ|sL zr_{1hMw*Mly$wnPk<9L$mL%z02{Q3Y;6aL4U)0(*Gyx!(^^7~k^0gl)j!-Rd5x`N@6KrZ z6h}r5K#8O)?9H^~U*UZ$7^oU0yf_+q!ZJwEyFUc4h@iGaXYjQ%8PxfKqN5akEFrhQ z`f#{A|5M_7p&-YmFZ4jVKm%l|Lknuhz#@h9D*9Q%x!)1uKC6I7jZ6rGcy*FCB_R1N zYjsdKGaK_dRnt!Rg|9IOrh=tQP*C_uOv_0FwOElo`9b-5bd&GfT$TBU1Y+yu(lsSl ze~uHhRlu`d-Jw3`?O=jb2A6kcAy0>G_Kdcdccl^2NO_l&IQ;t)dGf>7)fjvwlI7IW zC+_XUv6qL)=IStP0|>C%0QV?*6{i)Br2m-&YVO>zAK{($JrI6cE+{u6IP0z+y6E1O zay-Y4&^dKf2Xak45{E@6zh}bWBlQ@cJ_I$A?_h2BNp!R(^CV%yCHm=uuF2U8kXVb-=ODpg;`s#wg zo3azqzNpFgxVUnm$C|K=_~len5f0n$g}Y$oZV;XbS#lC;c+*K6aS%H2>&b z0Sl3K>A9-s3LYxSSHM)wHlvSmI(UrJ;d5Xk2t1;7a55t|APxpY4(XCvv2n$k5)E!3 z?k^vq?j#$-QNXe!l}IFf4Qk7uQ72POub{sV1se3?&?y`v&8tZPM0OYiRQ5udm3Ocj6w*BVC_3#LzR!j*F|y@ET+qm_Mu#5!8+4{Ck#clLx58HPDO*ElMC)ymR$R0b2x?v=p9X8Of% zVMZ_!Mo1LU*EuN?vlo7GrMNX09a#Nh+LEW$b%pyB__g{1d88&vQx&LXPrz_vl?5DT zqosN5_0D5k15fggQMM3%r!Us%Moem_CNOqIQU@(k;hN#imKxetGvsP$?MlIN2ixxV zq@g&UBHpL7YPRC5%%My`5DFKO*6gXJZ+7lqfxkfzP*&(Fb!f)d!fvWG#^_6eKn1KW z?y^^+&itI{C9Mbk0@6wNhr6Fs7apenGX_uyM$_x63exdZ`k8skHESP&GG)1vRIMq4 zrIaxmdJ^SHfr|x9-MWZ_h%e3S+%|vhkJC7L;Z&JFaX9M$cz~%9Ow|W>hLgS~=3)?! z@dX<9fv%s;~{>;fe(p?av7BOB|a>uja&VP3_&m&6G|c+3B2#%*MuUKd{3FQ^%6d5`T zk5@;|GY!7sH$a$8BhfkffZaL%QE7v5Ez0&+2$j{@xD}_XtB+Fzwp<55vKt<-?;-WD zGmene+KjDo{4xX+k}m4lO#B8k-*wtx9%&4~9n_6RQz=DcQ=$^I zcV%%8hgo75eRVN?caVJ6BPZc<6NR^yjRKy4b{N^iy&p^*LV3wH!AMJ4sjtVH&>I&6 zSum+@6<^x-XCl4y0kU%cXyyNE*%bJpA%bH_X_^B%qwJ(Ve^Uc{@BXo9`uaWsyY%to09@UijkH!}Jqq_&xrS-^OubR?s zaIVk>23n*EgEz~mMattRse^fAs655Jp6x>*6(kpR%6uO{)RV%R*>QZR)`we{WDI|@-QHD6sWCvDjK9rzqE@Qxp1RfisSV6!I~o3 zURm)HiyW$+msdE+!S?)=pPL~CnL?S*JlAGU7bYT1e6G!b(#460USp`=N=9;|G`6<{*jZ%(W z*pH&=&&`Ra0NxjEkVU6=Q;knezi@gedtM!Tfe+LH&f|7d@F zKy|}UVKEFNx7a@US-Bm~=VewRReI0(Q*=A80KexdCx$t-Z=Y*w*^ZCBV4!*yI2(A^ zWkHk@6vH&CFA6moGz&ax9;V#l6~*k{mQ(o*VGcKG!-03ZxP>6K+) zcDpusHj#il3L_8grAk7Dtc)B;*{gsDALopymMA9N1p1}v)O7nseJm8q76;)3r z1909I1XMM;dLOhl*<2XNB_Vq>i>`Q*7ysd&!@ZQ@0QU}hxHZ6gi2cq-AAi#2>L4t0 zQ=g$2%rX6m)EgyBFc-?hn~ygWEO^1*<;@4TNu5eTNA`K&3s;_<#3FMlQg}Dj7jNiu zf4AdEvL6Ym&^Gur`q_2fSq^T6WUv(o*m)eqR>DJ6Uo_`M$~W+OElR_sBoKAQ^bnUL zz8N9H7!B&8%C_N+Vi{0dAE#EA;Th!;?V@Fhe^s6EAOJr~2+%Zp+Dj;BbkQ@BMohEu z0kVTSOF9+@eFpW2IAGEL6g044b}Ob*abOH(2zqJ|XX-OoslNCpa-tj%?u4qJ#Ho!t zj0$4jX-bt%l55_>-uO1`Fp8oW>!I+p#);n%1$>cu{4+^f@)S#qTOjS2XH2g*&qcek zl)BUXawC4#0`rs^`tFmqc30e%S}6u_R@{=Akf1Fm6BZ;YUz6U^?}^Lx8$h7Rioa&DwS+dr?5 zW$|sk_l;Vc%IJt8dTNA=ph)B!FXRTN%5y1W1$>)$E}@WH@o=V&B6f*oj$cTwxY~z- z>Tc$FaY*Nw=kC4dF*47gU6(S)eC@u-ROQ{7qrDzvwyY&h*K8UQF9r@s_wFw2lRv4ZrW(d5a|&s=>fzj|bUrAf9ZIq*!JFp?4+lS*%)CzFfLaGYr5Ccb z26J#mVoHYZk>o@n7}1#gdPxt4K`BJ7#9Qg?G28;&Wd7Cy*OOD})%a@u=25ARPp24i z;o`UBdk+B`o8%P^MzY}+Qw-;V0-SG#QD85YuhF;a-{+m=HV_DkSnb`ct(ZarWQ}VR z_FbSQ_-|Lj|7?s!XR?oAEb(96NAyZQKYCB#h})>!b+;RCB&%+)n(B|tSc{9kYn;e> zHrx&#NiEojA3hXlVk8S-H8iSC|HB7Q5%b;(D~?qPQ0(@bL#$|vP=<9ftp;9|iUWe4 zF_~~Xj6|#&jn<2)(`l8bAJ0g3MvpZ~(6RqMSQ{mYmMk4tHhQ2)^YqC~o*G@^JrF)5 z?1LtFffEJ0+%N5WIjkY@CYq-*B73v!FOk7qrioZsi*dllLgw#{g%B0rrSM>$QIWAJ zK}3T|bi1p3d)|nJo3JIaqkJo&7&J8EsM%Ty*Uz0V}ZG zLDN&uKO+K@V4%u(PgIYnY|-hX2lB=glL9y3M2vgR75Jre#{3IjMk|1OyHu}m;EhK> zP>OXrG$IAVV%?p-sh<(lhcMuVP$qV+=8LX z2-P%m3|WeEAz~1(2+7Ia30fklUk$wKMEn=*9q`sTlM1$nVT1$EQNM#8jz^zzE^^j& z;)HP?zra^P;PpUfxG`l+L?&Z0!bslVEYjsUJp0jXOF@hT$By4jO)*WMAi0xm{FH?B z#DX|*jIES3@E01W(jS#grE`BrwreaX6niIuD?WL%Gl$63%POJQMcF%vKb3SJ9&{zv ziJf@w6P}nw620$Xje_8_I1#=W3F+xM_yw_KLV%`CZZ8`i0U}F{#0M3v7p6awp3wLF z1s7mMwYBXHpx20Q)u1#&I-Ph;FCjMBMU&MUHyn-^h}fyQ>9c_#Aqsoj>Of# zqs6BhI5ZlS4^DMbDF<7s@KD?Gv)84(`?GUHyRK9GJoa@F5@{!;uSK*9Xi*Cis8!-W zm?`+5=9z;3!ILits3u)Vj6x>-GB(nvB2NwhKl@q5zLPU7(fbYr>;(;~r<69AQpO9$ zpPU6^oy0i<8DB`)ErUp+wkZ5Uzw%wDwtrp*;?&7AdBe!1}IalxzPA0>}1v(YMV+GN2qUt5+>&`U2 zmz8>3iziLEvnGfh(neH1vgEzvsx*&I4@UqdXoX31W2|WzS@o#*7hrzq<0$SV_aWI= zu=SVby>`21x@y^2qm+v^N_jMjen9H4=jtCz`%<|dpkH(5INdJKyI!Rs4#iNQ8Spc> zPSa#}^T5D0`a-7ZeFBjzR9n%D*Zr&+piTF*ch78Mb!2VN`jLv550VQKCcV zzbH#Pl6fwL=uR$7y?T15+@V4f6|6VnP={HJD>h-2#Oo7ZBe6P~!#cYdS-C5E$mQsB zV;7CS$T}yQw?j=-LdLTxjI&Xgzc8`t|u=iEK!#W_In&9~?=g15XF=wrf0=3)u>UhVwu zy;@>!yz`CXB=#bFwX?^x2E~*sfyWVDws(%VyAk7)ZxoS@ZRxA;{XNl27xBkEbA0T50yz^3G`PKLd({2}pSnARHj#dl#rj3e zMK?kk&*4Phyyy9(NDorBZ)rDJme^+`20O6@p5f^6mW*%VaPKX!!nq3uE~r-a22@`% zh2U!8a-cq{<7bp9`gUpNF4HH(j_5NC%%Onw_~lp5D!!UZK@zw|$wJ^}9}QU5o+Pu4 zci$rpNhbiqOYk67ongX~vB+3BlM&Xw%mc}Fv#W@Ga5-?zklMw5Q(L%rBnD=m{PW#| z<|K`6*RYC)*4fxx+0c`uZxIz>Cu1FyKGuj9AlSOlg9FW%Noa|fDIFVqhdN-tx)!y?d$ZTGN8*`yTsXR0Cdzg-BA)G`U_BHIkCMk<4|(&HTrV(w@#Z z6cVYluP!b^!K8e&J9Cu5A&Foc@sE8+OOT5T{I)g(xI#a@-ziT5;4}{7sfiNPCbV9> znHwtRms-#b%jnWLV;E62vd2<_u?HKvuaz}WxJ2b7r8YgIT1HIh^~r<;nVBxP{JWKY zMBDxq;4jExwp}$IdT@F%ZcjW&^vulw*VAJ$rk);uz;sqi2s&xsw z2ffg%{-rsjbS61!{8j%6Bk5hS3V2un+x8>pB~^`cF2u|fx5-zZ+>wrB^icawaQ-d% znO)ttXq?f#x%4x=cg(|RfG+5*Z2$o~XlKmt7w{Qn$VWb3!6JN@%1_$IdfR9MxV#w4 z@a1zVw{HGj-YPWs-*Q2%AvE$h41&jD5U<)q8kgC|Wi*FHRz!Rce__mSvCBiAMkycW z*8Bl?@5Qw7uSZcudv0#T8}zG7h#@2C{A!Jr~Squ+xgPaquk9_puH z9+Y7e_zHnh%HFvi`oOisTqy?XI#`k0>vkQu9as=EvKAL6e3)vha933=Gk$*5R_EiS z<|9|w+>R^$uyexnHcCKN0C%EZf#DBFW%~OC#3EY>JTs(%PUAUhgFYD7ID>#H!H=Yq z#`({fqKHnx3NcYM`PO)2Fb>M$Wp-1oW=PW6OAq?09p_H zM?GUzB~qRO1+JD14P2H~7r-wF=#ZZyuNl9AVjAuU#^twHK!Ay@ako!j@c42Wj6~z* zqMsm!=`cnRkfdcmZ8i14F75@;%aqA!P(@)eZ+a%qBe}d+i3^h9)^Z6^Um%{VoYE%* zp)x~{@ z4t+1?u|M-mEt>(gu}MZHx{qbR^`6E~|EiU0jt?eL&K})9LimpJdt|X#S$3N?1h`cJ zd$S9BnZ974s%6P7H~F#bSWevAc*H8;+2fJtF5uX4e2kfjLE>uL%qi;=E9zsh{fR9e z3My!6ejUf>uY-_550Yge61?>f&<_SD!|``m=0HSZZ^B=5wcM*j3DM!DI%sa_0I-FY zJsK@rn=h%KFa2-47pw#xkG${6@5}Aqy1nH0dS|vJ%HwnAWDkoERLV&_hKS2RCe@r6 zjQfLDYJpXSz)_CrL7Sp^*R}t|?`3gciDuP)eFUlaDqKjqqL_QI8`~hdEaA%MW!Y`L zUPO8l0yJ@wcnM$g<^I7B8|zG%Qfl)Xw%}(R$rH&z=ccYJa2OH@zCOMa@ktnP!w?r#eoW^8Oy>wSS5ZpokpH>7rq_u#_}(|Fu_>b? z4p64n<_O%5UuSbznQ(7*AV;t_^pe_}t$*TSi|rj~95ks!kV7@PlIt0piE&{-TPH9s zPF{7rUCIlR1Tdll(%$GDAw+TY6(>3*VE=9^r)iGRBA}cLGu{onqwo5H2*DgAw2u5e zm%`CFkq-f`$8~mG;bU+wb#kXk=Uh`nI)3ho6lsd2Q{NQbnAIlJKY?0a?d+5Y#3*~+ z!P(>=65l{Dw9E)-O>6jw=ua%&AUCOjH8pV>yv;5b&~fTu>$n3%$65%m{^SSucV@ng zXMs4O^x^~+Ro1r#AI6~~-LQ(o1IWnqca|0o$%+6?`ZTKV5%&*hdvH2Ko_XwiFoDEI z$}HMlHo2JzNBM%D>q`?&SnYZla5Wnl=(U8@^%j#TXW65CbU045) zC!#r5F>+Q)zT{+WHF5;kYTp21R>F{A_gbSIITE-pDC9XPh3r3>^+0$*K#<%v>a;wmE zabg(9PQf?e6nujRRcX_40=;`H6jU#X1 zy3wMwV#rZkA`C?vreNv!lj!|t&~%5}%*O_9MDQE1JHw8~6l^5If^bozR;GzE-Rf5I z54H&xp*m+IR8>QcA~_c}60N-BC>9SB?Ta5dFBlPs5pebpM&_zIcxnaeg@5 zNS7Zk4#X2i^osd?bo8+iSF*k&c*WUx`pnr;E&gQ(;1<8ZI<}#G>VATp&*2buldlPj zzQ9TFy*j8P3^M0;_bPC391gtN1D^>B{#+|>pgIpC(OL^6dDO2;xj@Dz!Z>>bhhT`T zP|x*tV6lK;aUUhN#HD8ia6bMR_k&N2c+V#RM z2n-py-2d0du}I?ri|rHGete)DP+c27vXKw)pLD(LdW{nD(M~K+r?_~8GVw7_0Vg7- z>_yM$+~bv`7RoY5;DF+^Z(sCw+)H!#_zABXIt%1DHVX${Rm`OPw)~EKOvozMX>C;Xk5k*2cxCA8h3bMgP;M|BTo4 zRZKW&j*$h#j+@ya8PnXE_m{)a9YiWLArvMdwq z0a+j$Xcgt2CM3|zSvnoJ1Ezgud@$aEZ>e|X=+?s*M*kU~ajyVBfl4x3s#$e(U3Pfx1I+`GIooPZR3X{;==u)qUnSfN03hqV`9NH!_B zWRrr|b0m~aQ1NG~|1f1pgdqf2(3!RlTVDJ|=74b{!M+e9N}MQPKQV2wby_Is1_RY` zZ~`<-ZF-c2`X9kU{f~HfRlZ8*AQw0iRY3I19IkP#X+z&Ew*~)-4uA!FF3Z0g^uduQ zBR(O!Ti}mhSEy_-o#FPM)SWw-M~I+=2;g+4WDRrO9z-g!aHs7^TeGx7Y0QN#_&!9X zAg)nYJL&+@x?Bm|&uat_Fca}qXVnPH3oL{xmWF`q=NiA{b>=9l_S@zJiNv_@6QkMJVz9z7go`QDKAHCUT#0@Yj3;gxXM=&Seab3~*Ftz3~&Y zqXz(HSlT4vG|d_!v|m5;hbcDqgD&_+WKhXJJ@fc0V`V4_xX4)XZl0;_XW zZ{LwE>Cf5cWt$q4jF?sM1_*1@J$8NK`qXvW)#+by4eR?lk)Yw%+RY%SZ4s3^<>Lcc zRAPD0Wlx;;i*Rqx_t5vMyI12t!-fH~QHh1n3yI)xS^X+LL|lLwOJ6IT*@_m8w|2Ei zVUO(zJlB*NO-kB@3F;}and93GCs2EB+k$urS!r7sDVLXZE+@|G1&BD$O3QY(%S)cd zONpdTur}{;sp`20@(7Hg#H*C&WZAWOZxd)Waf7q5Jzy%|q{6X|usZi)>F%s(9Iq

7K-lup_B)(;(^Au~sRrk|3PNna8r4OGXW@poUum zK_Vj|0Tmn0l?CINYz^?5^cFO8Q>L_}(5BdtSPI3%en8%?I}&g-Kno%&dL@lU?r+ia zFCB2jWc)POg!fd<;$RKtY>45IUWOOmX#9vTX3qYphL|J&TeHZNUXcYu5npV{W zY|sV*wi9t@*(c+lr93M56<*P+fM=I{!nH@s{z(`{yTRo-h+?h<3R{ zCz-kU#`D=*8k|n81HQS8=p-uLO&ep;epPLW#P@pVvH2|T#y~qvr20~fA!wEg{d5DJ zJBeu)o3+b;I@15D&pt2wfP3Kb&N!2bqzDRv2$=|~OdmlJ1zI6IND@@T9T2ljgMn(s zk7!`9^E0mTbJyR)>BwdfykNdX22ggs+1VNS^yP-4=a`$l4)`{C-=~G;4;iDTEZwLf z?$!Q2K%Ra^YD$GaL?1Q?s7~p~M~Q)qF#=VWEH6eFFmz%}3X_o$s!@nOHupf*cf{y| z&#q-a)kFT8^4P_Dv|*f!qsacHsLOl!(S{Bco>6OnSG8K0E7Eqr<4(onLV{to0%*;F z=Xt3~O6I=DZ_6rbI$p!+g9BNurR?<=iSwsuTxb+7xV@T~Qic_O0<rz0xO(%G;y)fG)PZ1ni!0;2n0-@E+fFA7O8x2KaVPSq~D3x zoj;JL`C1i;^9QFSgF6cj<0-oWXd5MtQ>z~O!+PkCH)O{JvO9u~n5qoN@VwVRaJAaO zPTww_xLv7es=fLb9Bph~%<@f@682)g@Y8uo+5UM2NqTc+YNUG=Or83S$df~Xn^kvi zKt{NEXTwioqmhc2b8A#Kwu{QwpZq7BKMG34Db^+R-=c}{xe9p3s05AMOW+Ea3f<9u zyB7GIS6xsE>dUZWCRJ=^)HqtEf9B zIo?~sfCjbG&{Qsr_?ts7iV(@w>hFgz^=KBS1Vk2;cw;EbMggf7yVK^{z<>0`jC_Gk z-`7CUar%un`PDkRK%=Q7a4P9gvJ?lLg3v88Lf_pQvU$e8?V z?MmR@YHWQ&6621qnh04#JVKOg*S-S$3Vmys+6BC?eW%QH{ncFw zEkDME)F1!~y>WH-VkTX8x;%lxQ!ogqD*f)bU`g6E_~Xt9BOt0iPW=0rr~iGn{M^SF zU>qvMmr#5(J8(+EqKTSoJ+)1U8y-)vf(s|m}7T&4RtgDeK?;b^t|>D1=n;I zyoq0gbGV?ro-c8r^r(B)sWS5kdF>)~{BiVFe4vB@ZK2eP9v}`}#TSWnWeTF!6bQgG z*G_4d^t8(Sk!$+_jD#59_g(PnB7TR0CI}gqOXyvBJ1bL(EpRRHne?!pnGTI!z?J9H zv1y3pV9^jfC(s(XP`;n)tK!}xq9OMkLdkn9)W8VzAQtMMfQ$p)7CYkTqy4kL$61{4 zfkj7D6k#-um+(f++;BNJ+ky_ia4W}zhqq&U2fi}Saz8?=X~{?xa1zE*h7{v+^k@z; zgwHnAU8zJ6gbhvvph~nb2@>Hil7+N!+=Ys{4PJBO_t)~}3{o9Tc6Wr9RqY7nfu)4R zY#SOLVlmr)U5LL0>a)yX$gZ8I*Akr%KOjH9P)oZ7_eu`%b=xO`*_X`w_3 z`ik2tqxl0xh5N$iyYh^#eUh}9S5OlKjxdJ5D`~%iEV(v`^XErKsS`J={|zsga3?B) za*w&e-xB3&5o5l-M9KL5=zk}>%<2ligUUlnJm%skQ+fbx|9!UY6O%~M_IX8Kaq74? zake`Vj|E&s6Kxpe{(@HOXW;D}B7%8KDaSE-jxwRIvDKW~Ia;0IJi6qeq@;5?m1wYs z0QChOk9?zSi)@o{qs^nZKpQIPw8ZG4bT9gC5vl&;K7F^(cGCp;Bp9gl?mGY;QkEp) z@}UA(D4c;MFP)=WBMugm790!o-jlSwVIp%8=ri#`w&0kaxd=JhKGL67H0GK9F98Vi zO!g5fo`RI-6w1_zWH>Cg=@Sz1M4`A(8kyFt#hK4K;5VcZZS%2FP{Tn;lkx~c=Ix0E zArVe_xme~Q3zvcCOEV!*^VS?cJyorE-6cs}!-MT$EO@Y9JjlRp~f!}RNzUuu=Z5f>zVk6t>T~p zS2gO4F8WE%55KzgPN5LW>6U5QbY$cRUeV#${otl2Z-E`@lNioCh$FU$MTwr#!L3J4 zgwwP@a`TW2h`6DD6W?)|pm;f(2(_`%Cd|cK5C!1lTJi8Y&JWh%7a^*S7~3nY)fc#7 zpjbHYwiC9;7qxdT6j65{k#?WwFpa|9ye7=|TH|7Ww06NSJF`up#6$TDWVLwwG5BA` zWKM@d8M}-uoAWm%nlQx}<)fpq(%OecR}v1#OM@Dd6jiergABFi5NuT2EhoR!OqsVa z3-wbh^0G!9X>eZ*GpNBRla{ETI-@n_PbuIDlPmu6PJ#0&-VZJx{0_kr`xtC8F(b?< zbDg=t6iSLDdnH+tY{{aegEWrT_d$TS-Q{))bIPvP8&rh(KX~?p0QG>~Vb~jSR5JB~ zDoGGO(^s9(AGF58sxU&T^ffVdCNq^4zr6!g$VE z3*wK=^S4IMSGeQIdL{7W?ekbBW``sqeJd_?hGLJ|Lo7~*AQfQ<#s@;EpvtSgj@C-x zPKruk9M;I^T=~f7yt6}CqG$56O9P4Cc;m5QC~O1;p|JbT$`{##KjhcixBu#tbbxi- z1tK5~^$>4@tANKvoq!%M=xy$$Ok<|sppjufYb{~$xAEcrkET))J7|M>y-m8_cCGZb zR;e?iMm4dm#sqwrhXb!}uX#CCWN3G5Gqq$;`e0PkIcW690jDV9*b>;dm(Y%ytH!QdXJL+^Gz2$OhXZ2Q`c{z2h z&U!)Bc8_CS!cVnpM7vRJIXHHTRD~k3MBhtu{Thsq2o?GUx7g(Q;SUL1_cEYr=f&^EXTtX#xRp<+ z2Yn7c__sezP`9W_>Na(Ux=Y=o?o($akEqAgPFaF1Q5G-b|L-)FBy(&|DB|2^5Ng)y zujZhW%Cx)YS5AJ#zdybaR$jyjYHFY$8P~%C0c(|zwqO_&_4U%h6w^mU>r*+XyQuC| zE(|yid+6$T1k0N@I-?KDc_l z%n`6Dm+kKhA!g=lAee`j7mN{x$3N6x)c-B+o;%tT)zjCfnhBnj%^H1yv>oY5Z;t3Ihl*eLNfD5w{GkLHMrD}#)=bf z5dd5#)Z+cUps{E2CuA>B=q;8rm!ltG7Cs~5AWU}@#ba60H5P?#smRoP2;YPc;hSUx z>G&oq={ZB3g1>w#106v@V`b|yBT?SrBM$0|lN`E)p_XNs$!Ya0^>lcdnezD))g&Nc zi5gUoKR$7`!V?E%!kY2}Yh&lH)KUbF)q5c$VUpxgWjIFkG{^qdR5HoYd7mALqW6|O z+7Zg|-;$gyQ*jNPt@y&3^F#M1ZUPO|eE8RyNE0Sf=-Ps+Iua??GFM zfqFJn@K@AD3j(S_Z;~!=EPu7H(G-S$!F!=$$sAQhpD+e%!LFN3qq4e2Y35MpZ6cmJ0-l;fD^_?Bri4Xp8;C)pgf+0+WKWL1w=o;D%dB;^V zCcSi}TF#r~0pqlj?VQP#?iw}{b&IvY&%O>zwhZ-KjS<`N%-9T;k1t_KG`)5!yUUd< zj`$0O(4i6C59%p?Nz{YH!HYz z)^l5o*+zeqV>y$1c-X;Se|q|twwnM8g9$S^V({fLMe_*fjIryaw*4bP>5 zyzM=VgMsRJAlbx@*Oy9-DeyZ)kST&>XXZimG)9B1Sa{6qQlZB@Q%@!37OL?(3AR(i zV=onKiK=b=3eUY@`#t2)?Vy6>k@a{LAk;%sCDUDSeKkI;*8$&(0I_a4n1^$cnPd~s z645Fk$2T6#gRU?Sy5a*f2&ih^NeREm0DX>$=b5dsmZ-=+;iE@xqr+I8&=L?;2RPi7 z1B+;l6O;sFdMh@nS%$j>)|XF!kBG4)lCqN}5~7c{0R#$Q>c~HNx+Q-95r4dn)>z^6 zJT3)>)4SK-4knh-O5l;E5<;d&d+iuB5u$cs!tecTzwcBI;>}phpdavFd&DXjk2g?RkI$>^&I*xHFv(D$Edbr*Us)9ww68h! z-6*O*K9)v7^jhH4#2jX(*axg5M!o?KQByITKk?|xgy?#9k8r0fXd)J2{w*MdV{XUY z1U`Kh^n;1eh3L@=eZBqMYEzoYg9OE2*6mh43|xTOf__*=8Phoo4u0z!Ora=FYK$;q zwf8{4>_q_HrcZ|H&>S47>jpyV!?#IUc#?|Omt_EI@yt4&eVCVb49?VA!$_-1n z{U@O8^bx}#^ILh+hAq09IZ#EPXo%~0je?DGd zc`9`0;y4z4zIVZz49RXurex33+EvleqYrYpQLrBPn^&O?^5g(_D$vKsGSf`k&>y@3 zLP}&pDsgteK)vDGFJknQS#P7oCG3LaJ+kwV3sEx{WIA8}pdz9~jgP4{;>IE8iry;e zOu9ux7`CE?fZ49);y1PZnK!k3sMJ%Lz0dMU8B-l+loO0>S6k+UPW@3TYiCgaAttky z-+)T5H}{WZV~LOgP)=3uqR$>0?WG9vmy7ul)_SmIOiSgs zBsOlOo+2*7zfX3&_qQ{BrmbiuECVjs!NF2SzahPqKdU5)fXjfIAR<_<(%1j0F$rQy z_Hy7lENzmMy7Z_}jW}^b#-kBAE=jBh{sOEatnT=%`59$;Vfr8PbR*gTwt{zm{~I&O zMo;w9Awpz|q`=<3B0_c67syWn>px|F9CaD1r?WPI+(~JhV|j{sv_)Tr$-D~fO5lFM z#ik78%qML<@E1%CY!hmzEvT!cU=uDb`2qQm&YFJD!CA>fvhh8V1b9`K4^C@SJa>6& z*~9S9ToMdKr*|+=6>ErYJ$tW*f@z4p#L?zpYM-nFEOxeoJouP;tT(!%*nKMqZ&X`K z>PrH{vANyX4G6MR17?KP;-dmq+nnXu_IbOvK;J-!=Y$$WO$; zj!gFZPv={{;;Bwt(`ng^2JfjNc4kpPf;Ybl;@e<(tP~ZJ47HU zESSbbZ9;`oe_(=#}e!{3+CO^1rGCx@8hwbsZDZ8lj!PxKOA9k{!|6~3_ z<@eD=f9!1Nzn+JAND7>L-`UWAtc6FhaK~%ThW_ThaRTY_KtE?gfBxjwL)>xma5nV4 z6FoBuBBFW4rl(h|wd(2dhBHs-=$=cj2#vSt@A1d?&<9?eUQLV_-Q4beFX?g4Ux@`O z?%t|!O*O>eF4ULWKTO{Tv4%qTB!MXJsX`pAjp-$$fgdN7B2t@fc%GqpKFZ-Nr?1TBtu@8tDydDJPsGM+O*~k4~nzCUSiLSJN^X$IRyQxPC zX+EUFu6B-+y>jK={>yI=BoG|{Nrb}sZj%fi;FP#2@*Y@o)b*I_ao0OuL(CFVbqC$> zjUf1ec9DXJqWp6c@>#Z&-zT-CBdmR))H$YhZc0%$wc|b9B+=RFUMlokp?)i_p!x!N zvS+vdA&r(j+#e1jArECxA`?)m2fHR}sllBie?;k_C~k9a=zR1pEk2H4af(I<`!1GJ zren>2MQ=j~z)WDTzwHIL)F=%{IilGfWCw#|cGDeA1q4Dc#8H2Qy_H54bQEbYcrRR( ziIC>kTGw_42e}sgUO{>TJ9b8Pp+4u=IFnu6O68KO-*8DqtO<+l$+(M6|IprpCk(jZ z%^AHk2lrPeN~ZUlBqu8n+8xTY8-Tx!QYx5kkDmX}__loyD7+{(El}PBFr^sgRunSX zcGeAl_m#jcMzT<#!O(!A`a{(9gC|EQT$&XG-khL1XDq@=)(v)>kP3eA%mbmV7h4CI?^a%^6Auc!CiEb_svV{yk{ zwnzUPeUJV(crOvX>cCFPt%4^6x?DsI`q*AcN3Y>+yoN-{e{}}>@csMSNC1-H6(@E} zDk5S+FfeE>KN{kNPRvuc{Y`g5_-UOr7@iF%Ctl~|vMa^qxl$quMg#Y`k^@gKo0+@4 zha9*WEVI*y5kd-Hpo{v?g2nXzis;s3eF$(3!_5bJnZ_~RN`~*w=ux9il(tezmpz|D z#ZXQq6zed>t_?RYsBls#EQQ?TzbB(-J5ebO6F?2KMqNcDS4Wy92Vlx5$i6`po}G>) zHuDw0-Kx)RSL(l9rihI&(FEw1udAfI-Ex~Y)s56)fd4X}dd3@)8<{KNPB)*Xh_J^| zV7T$^zOe`Y&ZF`QmdN9~&SFsJdck1Kt>y6u3e(~0g>&7CA9XayIxQ(H&QTCQybP%K zyYRJlV_`7VLmVkAhW;=5_}GI9H=KP0168Yvz-+S<-Bjd5j>YiWRluW4HyU8=QC$ka zKJg8gKR!R(@@IT73%?gHpsT(9I${ZEHNQ1-=ee6ZqUrQFpwUWH96^47lYwc+)a-x`@HfO?(7worn!pxgt9C zj*2KiZT&{an6jq2m1?_Mc8}1?dl#&>>4h&|?N#0$8PL=Wo+MoKLV$W$$M1<3J^Uo5 z<3nv#$x9{YE)?&MG&Bzn78N1o|WI;LI^ml5zU<0bF#Q+m!j~6z0L1mfYxzS zdm-XMF!4^E7{V8Z)>dW}*~KG5CskA(wd3CLf5r9caM2rO9x<~uSNsUvu+Tz^4!)g` zyqy>n4!ow-Gio8U`D`vdo&WGB_&Bwp*%yq;h0dsa)rGkeKE#%^LI`@!Hit&fx7Cw| zxPky?K$*X1SshhhU5ZV^-!$(vXI0(qHh&IWiAWiUr5a0&*ks!dk(&%LhfMw`L_G1~RBl%TX59sh%qKXWwXQ|4pRjp#CbcKc-d z?D4SzE26SfDfkFt_p>)Z20Arn*0{JignyStV@ku0r_uHGk2c|(UJX2NY971fxaPXv0Mjmal&JZ5@Q$Q3%_zsG;YJW> zt?=yE2rIy`FC(e$%-t*T0(@<8h9hgzW*>^mAdN7D=?xQ--j~ffVwgylm2l`9%Cg{c z^SY{a$F^x%?>x=8`)lIV@nJ@Fj9#V%PxX5#u2KKrD829p^6OTq5v8GFm!#%!%X19b z@du2xPkBbGWwYVyQ!$9K9PE^RZ7OpujY8n84fZ5}M~{QcFW%@OM3qHNq6XYA<)nD= zSXcrWhTfmHjGR>Cq-7;=x3@g8QhduKiTf)znpln2-O}UGbq&$CBW?-Knop-oYJ(o}~dA1~p9lI=*QUOs$42uQ3J6>?B~nY3pwMa(XtBh4VI zxX^|w`_MzwGQKd0deM9I7r5iO6vA*P)mbyCAPy>FF4Z+AZT80B6A{HG5NuOSIOW~u zYPDUkkHb#79IcL!;`QysIkaZ&iAyj3z-w4~@}3eei6j%HZ$ttqP>NC7r;PHXmew#F z_uAF#ZIS{&3Y|3SxtUlE8eyA$_Wl_uxfj-T01T$x2kmd9C%uK9B$ zAK{fgZatk+z2Td@}kG#3}xB@gV5c+ zwPX=~Vs)kDi)k341{$FTpThoti3qg)>n-?`zGkHQv`xDi1RtjtkuwD| z_gv0Cnog3Ev0+e&Y!UmJi&=l18sNTzerh%F>=9PY**##O21|`wNDU06W>7D65DaBf zmvcKH!E;^-Ji5H+8KVrc{PIFI+zNMD#LR7F0f&7|t|(3Zzi6eoI)N}S@d5cSFqy+r z9WT=5v8{n811mf$Jy9+UdItunnWM%Zq2JvA0*0kcl2eKvMX%zt!V$Jv$T9FFbR&}= zLZaR7d+gHhqF|Q=&4C14j(72` z=5JEJI|a8cz)Mz(w8=nk5Ey}@hE{*;_Sdz+(PpP(=QPQLJiBjSe3bwlvsH!NU zXU-abPh_*MHlYj~nVl(E)_=QT8BrKm2Yem61~{}F>!)ru#_Dj{GZbj7))(Zt{K&8L z^wkg|4tHvNGrpnx_N}cB)CVDw?v?MIZ|e9bVyH-c@TnXQL}D*ex%RzpR7wDI9n|jp z`E&|bYXSh%uWfU&CctSBz%)o$vpZ3D82w3?ao-%p%YZcz%oU#rJ5$|&Dac94vq7*E z5Av@0UKvb1M3ERikHfBIpd3^yuDXzUdJL3uQBXpb(rp846y(4-j)K{o6z|tzg3V~A z_>0t(Q5Hw7g1J2X{LYOP=yBcck*D{|r25myxd}?NYb?$8yE2Eb4eq5-IR#^vNtDwX zv8e$Hu2W@`1=mS|{czxQ&_!5G)}a3c8uXvwFBk^g8XQm7odb}{+-uxGf?%&GZy*B* zL-`L&GfaUbY}COB|M$b!i^xZC(xxchEgQP@_{Y2!zC&R*YVDJ$BcZsg?O+#ZZzd@y zX_ppq^WM!WveqH=F`r5fcFtf?Otn}=p$D#jJ#Ynnot8E$MpWQ(d&aj~bM`9n+4L@0 zz4*dJc+Rno!2M^Qo+c*0U^+DUY0+#8AbT^SvHVGTXVZ2ZFa!gYL4H#=rW@B;DRIl7Q8cj8u|ARl$1|jz-yOw`fXd^N>`P!LY2ph3eTDiaaeLQ(r0uSqP zQ$<(PHN^1{7jhzXpC4HXclph4Ego{VEV2x~VM>Oc#JW-^%K2vuNAJ9~!-J?4tN`wh zy&e0)OrlGAOS_yr?%l6aR(nrUG#+r}$y8~nX*2nw$L%zXYYV)RI-|e9Nz6KD+TLv0 zSj&hw7Ier@z!}L*Bod%y#97h~&Nm{FOePH* zBQfaF)BLWi;COtgmt~t_GW6gncio*>CV!^yEvEK0+uhRi`%*~f_Yh4gPUoR zvxS=0)`iF}&ffZ9@3HecNR)FRV21)x(EnIpL{&Ew{1+kYAnN#X2If(wRBk1D7c0f& zlL_+l(J#et~TG0bQ_!&WuCym6~s2$=38= z`Hyqn!3T&Hw@1XFxQAi5MD^?{bj44$0R#xSUf{o|hCZ+w+n9;KijL9s1Gm0~_8cD@ zZqr^g#x{Pp-^JP#S3Egh0X{Rdh_$Bj{Kne}Wrx5ku?xPx2mVX(;^ zZ27VNKjD~DWH;-kYcmf75FlPBiBs0Mo4BEIidO*^sP0Y;7h=%L3hg@QggiwFop@q9 z2DGjaE|fEU-%c*M=ZVLK0L@F*+DO8N`PoowOyjU&z8{d^)>${4LKsh#Mj7#v*b2h! zHa(}U=+Ot0nu&xRM*{Kow+{I2p|fZ)ctMW4J`bBi0Ox80C){=&%zcLsdci#7!`{1H zGhMZy?w)MN{p;5VHJVKz=qh29D!6Y~KrdJUz3_|<5nwaL_?0FmytXJh#kHXR68Uw7 zW`O5J)kPW63laju^nRFoB>hi3299upByo_^*JDQ>srjgE_p=IJAJiPuXhC~(J7xTT zl)ZOc8|j@t{+dync|;Y}bT@3=?R9q7Te6!?-)xdivMG1T<&u1pT<-37sh7*;vUeF3 zddGBAhTcJ7z%<(JZEf8Cr66zBPvDr5!z*RJtt3-^C)+lD^u?Wie_(fH)vC*?0vw_XyRskXA7mG zmrDz!sHz9A`tqdKCM2TeB>|9rRN#&LLG{1ecrttJPZTznJo$ldx-mR`E$erB$%o0o`H4&PJ2 zSa9eIHL4q1N9b_&N3pmNJ&$6Y$(mnhmm6Of`;aBa5<17^JDLnH3t*$ zs7$V{sqaFhw*r7O)Jj4^5HkFY(cG*&w6t|lU?>s!Mn+u!cabW{LEuy^eduFa43sBo zZaYjj)8RNW$sHh}*Lc12ZVMZQ|BCd(PXixQhkO%JdikZ;GCZNSbVdZb-N%9F9hn7& z`faKP>q(@Kva#S1;0kpiIkp7y%!X<#{YGqy@$%cPV2>c&w{6a9e| zCucD7-T2c6TudcrO*@FZF-<9cbvcRc9a6^)doogVgac0YQjw>A;ZA<}SU=HWtKRi)D6_EIUbt(x@o&rz>;gapO_SlTif zka@xV!qdb&90o}i>XY%x4?+sEK7<>fAFwgxbTHw*O4W($G|XN$)${>i5n>?7vxRuv zGtGxElRyBgWXQ2PW$6R7`gh8DEr^KzeEjOT48^MO0BT9C!{gk_CK$KL74L$B1(HI^ ztJSG8)P-XDobm>w{C^%C+7{t^r3(OtqA)6kEn4{*KK~gCWnfvkj4^A>U)jyqlB{dG z8Vk~uzhQhPue05Ib0KL&k5!tL8S95U{1d>F*T~CwOO2aYcyG&H45pgAC7*Y1&dtHz z(nmYAHlE5lk?S{Eyl@CPJcV^p2mOa5He4arkAE#OpuAD{CnT`vy}>em}D?Bu`bX@c>i~dJ; zs0kS1?sG?7U*j4Mu{ORYr7+^zu7rdtd-%lsfK81XQriMS%4(uIg#IGCbdPM;<^r)G zjHg^C8^l$~@1yNp7tTHjzq!m-nL#5~F~_Z$hs%bs;C#ecEC9Gnd-6V)I%)z738uXe zK?o~`8g-s`MmywiWgG<_=1zcc_$AR&p$hQ=^aVE2j_8kjaq#Soam45Rq_rPMhHEjZ zl;uutk=z6T)eySSMg4j>jrM0ZA4`jWJr2K8*z+Cu_^heHJlJ(VKKWlbTnWCQ({qKU zP#C8-ARqa#LEqH{M`iT*Uf6;^1NK{Xt7ZEW_1Ce8z-BwRz77DVGFKj?LjqNZ6)G9P z&d}F@@1V@o8Pp{4D_K0G9-8<*yfI<2+CATE4`-jkp!tMiH?UiF;daAJ4Fjz(@}`v4 zc+Vj7C-&9xA5WV$o2f6OFl4LxJP5FUJ{X^&d`Wd*(lbieW5|73MuF7+x+jp$%ony2 zK_O%6>SIOM)4qx1soQMdNcnkZ6TI3;ojiDc;wXyY73^`D$l@Oc?(?F@q6N{SXi2n9 z!Ai6<68R!1f(oUe6cImZrr&v}LAwXJ&(FU~e;uB&z0?&vIZS_$-AxVy_pNRLa_g|O zWU5QJ7ud(Ua4mnyG)AOapzPrD@CS+%?-J)y=Q3y1nX3C7mVSQOABIK8fES15WNMBn zb-y;?$AuJ+fya0WR*IEjv5wnMEw&M-B3U}^HrsQaq-v1@G-3;l*H;xV?b=uOSez%O`pj2v<@za7fF>xmyh6C8TrRK&4^N!)R~z0>^geWFr(jE4e zjC4fSz>=5F`&86=vD_DF3m>&;vPRt6-I*RNhlO4%o@~pdn63rZ2PKBhAKgqRZQ~UQ z3bZTE;54MOzHc~`%ni#)&08yxiI<`Bk3{hX*qbBCmE>8ewu0o+P;q~*{^uxKjE4u* zPRwMbu^1~xX4oz+nX{?48ej#=0*txIe2=~713+=d)l7T=WZ#!|k6e#Jl=1mor7pQm z>nm=;sX`i31M?Pb6Bt|%lEUzh|J=DNYP3hkT}a1fMp-JpY@&< z-JO{3z|iU+&x(%4Z>Gn82Pr>Wux}W+h0zH#B2jh_I4|zvBL$5q3vv1;^*eOgKk+f< z294CHXjIKYiU`GNt1+v5bu3#vJry=3hUEfx;;GM;_#GwnA%X~37|DmE_dLms}+kDUBU=~7WIjQVZ(ehK*B4_Oy z`-iSz%w1&yWbz*~)tcc7J|vXl>9v6fz-c$I8TCY~*%M{Q3zp6mBaLG}VDr1*_mcyp zQO9wu3y-+qE(lc{0UqDzn zX6l#f>3pnEmGv7I_29!%za(!a`#pHj@*`NI+{qnH;&*sZ4av@}to{!|&o{gCz9^gC zb=G{1#?5Yu%**poW+{)so}$@^e#(n_NgaZnV6$)0=ZbVatn|#bvgq%A51;U})T473 zCq98sm><5}DTy!WJAzD5ljjq!vt7~Ve+XZL$^WyJp1}{)-+?evQvp*l?J=osV-;8y zcz*1qZb#Lq5)t4x`6*$vgbAZ1a4Xw+7+2Wu2&OP5nZ;U~Ka9K)m$ZxR{~rt&cAH9^ zlF2!?$yK!RHnluoa+v|Di#lKUQ%&ad;1yA8YaqmhZ!v!G{E2a=zW1 zaT+PJ9|N96ieg2H;;LPJm6(?bCK zD+$7+o(|Im*}`qe8mec7dW)&P*n#01ORsQ`?>XH*~;xJ9+3z<4aaY;RSkhAt( ztA(XX@s-ikuOOR)8}ZDCFl44Gil8ahF494`!!vvVAMd)yK2f=_7Thr7J+8?*0{4jH zrn-GxUU}`AX{A0975cM4Y#(z^Z_}mW$U$yl@C*02_(ys)whRESSNIiEWdHW0YZ*r% z4_<(H#Fjl~CuREUj3Znj{DEZwu@;xe8hTMO;iIziOk-%_1B_eigFa^LI8#OMS0XCF zLdxSyzS@YoQ3MeE?q{iHf$P;V*z^Kh?*3TFZB0Gq(%(MuHaxBb>Jr4uZ`_T%^nK)H z6z(i+x{S}aNUd}UgYoKV#HsTbGzuOq!SW+vOWjW438XV#o%TwJLf%l9D(=l~>47{{ zccU9rK9!11YgiW%2YA7DN>rl{+6T+$zJQ4Ih}uSw*>)&9FyL~u zLHt}2TulIeOKO$)d0ycOxXum(*K!|P(&}k;75SC2Te93XeI3T`v6lRWVw9wo3jG=f zZOn(6t*LSgnO7T;DYVFN2Lj%H^4Q8kan+;T3=vX%<7w<1rc_02Igt9LQG=S^#iv2N z>RkFX(gvn_ejGI`&=Z4Y52mb(kfrJm6iZHKKiIrb#6ILwMxxd#UEwzCPOSJ4t3C?w zkg5{3N3t9~2gNQRUGsVBkqs22h9W2>>ss$;c)briUacGA0i^5OG;&Zy#f#$AEIgYq zJl;^BOviY+Ilm+=!Y(elqzx&oeFj|O#qaclvL-`%9AW}dQZJkyIm}*ttlp8$>W>O{ zHb6&)sCIOrQowLK+Q`Kq+x}lAd z^LbW)McpE%U)BFNqPyfUFUn9naq0Jb>HN^;k&DtrLDrmiWn#A~(PE<%9#8(;AwSo> z`7v^m@WIwnZA$v;zmE*!_wK#)jhy(t!*D_IUFwQxt|rk3Ngag|(4@9SG8bBlB!v6; zWHp0DFAEb$1pKp&sq4T@v6S)Nk7k;Fhz#TI{>{c(K!AA8@EcL*=cIZ<8vyn$A1IL1hsAa#ym@K01%aXP@w#R_ z3KRN1hZJP_2KCwAZHiVPW$!n@z8MLt7GVJbT0Q{u6a^WqCXSh0tnyi`McIlWUnA7= zSc;M5#{{2&=NtdZx-MGFyNk|rGlJZp>)o6ZIV}DK2x~G$gBy9*WN@bO7G*?3t@{y? z1)-u$rAJnAD&n{1Ah==*MVU9RdLYSh04&-qd&PL6cNqIab>ZV(FpBa9j8MIpCJ`KvC(>mx8rTI1C4N0m*Zpe=__pwKP(^n#=rxeUUzZh;}dn-ulir z;JqE!XGnO=9s?f3&ZZ;;2q|3|6J@T=zYRAQ{+XWSyRxwHO=Qg;1Wss(#P%VY7e39o z+WQQLsqq2ewEdul2u_$iR)TFLzB~mV(;Hw9?V9EK=9%kUD5^;#JXiJSzDL<{i0tR< zz;7D@ZAt#9qd6B-8vbw~y*$iGqTXNK1gQ|fFf3iMYG<0@Wgly9ppn8?fG`nH_5^1F z{a}2_Uw=uHH79QzLm1Nje8WiQl=^&bnTT_4xXjwm*>4l*(6RE|;23Y_3DpOioYo{; zUo8Af(RA=-DO}Q)q?p3~r8uwQOUAL^SDb#8hLG&9n#Y;;oZM4bIB9rT)SO><1ZDuA zdxe4l+39z)x=@9tdd8-rIoA&UnmQ2GrVl_qWLa+ShWpUPMg`o6{DH#o8Y)Q+wGu2* zoua}h`V7)%=Ib}r287f^-$0wE$AKqL(E_P*=vFZOhW>BtVQ1XsOq2#ePy;>}($s)) z5Mw-xoW?ryX%B?h79b?UK$g#71zOz?;hD^LT0L=W9W7xSf7`BzJdQQ&-ch3Z$&>pn zEXBV^h!m)o)J7~o@m(7M@=?G0i^2RKAUTT_Odomwdz;srVE>xI`SagOEPqiCOa7-V z_4#w0?p8nb;)qMwMcHvpN6VO_`b51eiY(`8*$biX)Bk56dm5966QLrnW5ER>J9gt}2^)|3( z;{}KpJ}-CL-(kvPabYa|Wki`rFyA3oUccR2%qnkwtndb4)90+0@tZDFpY$ZLtj-gL zB+XbEwPw#>HXv<+s&?)>8Pdl|W>T4C2|j;eA_1;RLC))f@?dyR;D5tIZ^wjnOoC8- z*x^!a^2R9+JAp@c&9r&84>(3hO$W(5p4`>qoE973N+=R{+|G4GHn!uyGhHIQ5{N}{ z;mmUvq^Wem@(0MIA3PbWA3~STQA?KWb8pmK*4)66{^6EqKL;2KVTR?hfsS$+f^6^t zV9KeDEOp%CL(A@ARHz?BAqD`KL|fh(fwnXb>&YGe54Lh*$+wqO4ZcM+Q%8`&!pPUY z7;$lZegF&_wg;n=3J&h-!tR%q`@?*|57@+eqTIfv)ZKxZ>hHle$O+)h3tU04x2Zpj zX=PqVs)i07QWJk;d3I7Qc6(&{5)S)K{=g}n=O&3Nmn23hhWBWRJU%t>@o*yT!!Aa) zve!<_J~ovX_%>2g^1ss|j2pWq{eVq^H;S!~A3BG{RE+;83avruj!h43#xr~V z1j8}w#>vSiRk8UtuoU5Af*O{3H23%+40QnlWwC6we&Y*B!F#9ZlVHOS!VUHbUDbbS#|*LE8g3T7C~uI}pT3C={G;OBr}8e#2Y=N8;^wCX72WB$NF>1$Fzkmu?v zu?{BW0yzEl}@M6V#jRk$ubpA`5gSVVgl+NhWGJ zLdvF02ob2t_JarWap1{Q6oJuV9i_*pU<%ep4*=&mK1MomVRv3>wiHgTBf#~s>l*K@ zNo~7|B|T$8BrsL=1kyUYV9|=FLLQ?|6elNUwPOf$Taa74F7AImktjksY?c%^$U6@u znGf$D<5bqq$3R7(;&n>9+AEuJW;4WAgqPeF`j*9J8SP0<;ZS{}XKk z`nCdqv(AyHNGZP@)KBzWXwHPny9aq&I`W~*Dp7jhRFw!WLLmlL<7r*n;FqldARl6* zC1;Pb({z{O0G}i{IG>8IUz~bIA_u-dUra)lxL+N69Eer~d3z6MQgfd`sjpufp3Js? zrfD1y;TM!pA7i3#ookb4Q!Fo$qe6|dR1~&aKtm;%0O0h*E>CXOpNW|X$Cj!m%aQWH zLExM%=jjN6VyYM`45Xp`t`{I`y?4-_WCP_*HBjELT=4E^(M2L3J#poR19jd;Y#`gge(_OASsl*n)kUpPnCx=ak@ARCA;ldPn&JPAVgVKjF{qI(AH3l)WltUF03 z#LWNnO5|)Vbdt72_=yh_omakOIsLnYkg@sK#hG0lXE7M=9R}|3ZJQEAX1!EpP>&oK z@+$60yfuM(bkYP>>Th63;RkFV*>H}vp<>Z0Og2v_j#)C`0(7&qK-u1daiA_+JAl%T z-vw&Kb~ARw7CD6k=_#udB099<9TZczZA@WO>IR|pSjbR%i5O`Q9S5E~=Qb>bwS$sL zjU93t8LMG)%5WLLZlDXQ-Wj$E?(R$;Wp@%a;Y)=^?Z$h?g#f_SQ#V$b1V zHsC$M<2jum9_cIDjeMCIK=6ENuXwbG{vFaJ5iCr~2enEyyzMjBA;Ux5Y~iQM+PO#t*}IJOH?KIrB-@lEt(OQ%t+SBl{q5R$@naq$ZZg z%X4#fNL}t75TKU6{ZR7lSJ~do#N>thqFFz``uK@2Bs07L{4~CSz8cqREJ6&WV>m4? zfKe1B-umMq+j{1d*pf@hGG4oW5o)fxSh7E9A?m|bvG8p=3ftihmWx(~2mT%IMf?W4 zWM&k;c)R3Ruof&Z?plmQ9~$*Hm{{;z3WKuqlHleZB?bpta7a<@emam8f_gX#Ty-p7 zvKkrRiYVs@+JmWPv7p+8!|?l$TckP4wd!$c%tLw@@()i~{y<7|1g13nfiu;_P4Ynq zp3-{#TkvMh?3KozW9P;ykHeeX1MCLH9HBQdK`Wt3^lal97b;&iQjRgxdPLahAaLH^ zJxzf^346QBZy!T;j3buHe9<)i;;H&;a0YEF_M;_UAK}Loiqtdd)g_Lp3)ns$6U?z)_$$;$=Y=CE|(dOt#ka zImLe7%8=~$xten5{^)9$nS8-){GfQMqkx4CQ6~)Njvx-G>H#(rmI5_HKQv5@2~Siy zFip86Za+(Z3uWu60TBtd1Bq8UIg+5x+jfMLw(*K(kC)Dy&eRr`C$;2QOtpRw8?+0e zXE*A^bU1|w79F#K*zI?&$tec#7WVOFF=htI7Dc^633Vm&CW00^20VCLP;82kIEP)& z)BYTtS$vJURAQ_!oR#jOfMAs%{MhY;>t~P}CVzk6`Bz?EX!tkeG~_KNRA5)~0wMS6 zZ8Lo;A{RTKcS($N-1eA48&(prs2d3z7=mK+0>oW$y@+wpIVi}=JJR#I_IKcV{X7VW z7bT$6$WrdGuAa81FHh0m)E=bz!sKuHq^KVO#&~`g~hfN0ds%uMFVBXj}a4+4*L%}5;*gqXRe;kS2Pgo>R zg3wSbBQ!1!N1{6`bj#a?^5EHQgysG!opp*VX$blna!7jv&_X}SNjD-hh7oeu!51mFKR{+Huk_Wd-O#gv|3ZG=Ti`(OtLEGZ)F(YEmg4JRDHNV57Ew(U zV&Y-80@JYQ$rcP*@O;JV5}U|;^@m7$z>nJMRcPk*?#Nlsi-+8EQ+44eGU%|djBAtX z+d-?E3=EA&_zFe+yPF5HBXJJcN(hLsQSTc+KlQ(GI{E@oZeL+PZ~NGG!B!!kFHe!* zyQhzUW!=xFT|{o`{;u5A{hd<=wElS(&EHAIV@w=tMDC|J3U;BL7?Z?ec~M#y%K{H$ zFvSVr$=k*^a1IO>x``c!0b=_$eKs`zQ984FIm( zh!8fTmS0yEL%fts;Riqj#4UiNQoQeF`U;LrN{f~Kf()#B^_~-~Q+NRBO|L+&WP1gO zRz_pYRR&60`%H{?4Ch%yRO#rUBcskt?LzH`O!a4lM^ zd`sB;!~|(px124iQntAJtdm2=WE z$N|J$$J#@ETm!;k6vwdgKy+5u-`IIf!&nG=Uqb1Rt6!ucSnUigXGc8!I;wU2fl^ZjZn>e zQGrNUe&_<_s&8dHVJqgft)B38iyw;X5mR|I2@i=w=)IX>dGyrJC@pmY!A?1HSH#wq zvNCR4SiSf+7*+Bgc-``I4|>0hoQu1GT|Lo2G!jk3CHL)1J?fHZHLPgN$5uglOVmA# zNu$oN`@f)cdG=Scrd5bT#(v-cd8s>*hWwsJD`oC0t01O}*X^gsy}n@j9b|GH0M1ZH z5gqwLdaqZ5XqH=sJSO-t%U1xbW?3G6)@5uY2Vs+W0BJ1PE-KE1W6>ch8wiQrV_tbA z;yz3+%qG!>Js)s#^Td_mCK|=V`U6L3>A-Gb6BVEKgJ#>rzxk?aV8nCB#0s4VHZuhJ#F4$DSas$7vp8H?!Uj#j-yLfY|C_bs zE(>e!(|nPg`=5Iy&%-%9dbRc_5^)Cr7eqZ3m5E*J2)2fegIB?lt-|~~d1!VJx3+QA;(aNbtko=Y$grHE9pR5%zmJXFU)3Pdxdlhp zUNSf)@mEOG&xBRF&E10N!3gS9yOMRxcTgM@Ox;X*DTjbdAFl?hykR(AJe@Fp6*jW> z^CLNBGZMYZcu7XW4?|7Qc!cHNZ!1L3ukA7_zniMLfYCwVjG8vVg8vmN8P|tUd)XiV z8t~>*SVMkVefba;o|U_VARl)Fn~RQ@99el-`7_lsoN81yVhGp5xmp@7H3t=$Im#Rc zu3zG;Bly+Jf!E#Y#`J4A+A>A4|8Qu}OebEanz)X!k>^1{U*8MV8_YRYz+elszHRkt zcn5IU=m_Z1=@qc+a}>DW@f2vK$knRbc;UES31`3wD{`}>CS6RI2KvLW`xw}D(Od9E z43~ux{q1@k>ERsS2D_BKC;dEV)INR|jT-uDA11Vb+b_F zC)w@ER&fwGg?sV5-&?A9yk|&P3a79iuvu_I9Eiaas!!A&MZd*9g_G7mbL85%bNS%4 zPhnBs14u{Yp}vM~_C`q#wpvEN`|6>GJhmlPq;DT@#}Ep!|ilVzHq?!CS z@E?^J5mM9m4E_bd`Ve!glq@WaN`k;Ne-@^|9ct>FZG;2H@Heo6#hGns4oKk4$LGKI z%Ym4xQ!u9R1|Xl9vNfu%EOn7gFe^6uF)y#{j#u6Cm0u!f;6VY&6+&gZc`No?$V%mH zTHGYk@27nk37oC)#FRnO%}q*;fXmEWHYg>9Q4IAy`xXO$%Y+-Rg>;>+4x>lupS+R^ z!*Ub8nYrAABg=xn4iE4Uwo_8o<4&Xp&7Urzi2X%Ft> zL)s=Waw>^cpxZhtoNhYs2$X3kMLR{~h(w6pu+NYkQsM#;$|`YtEd3RPA8!|7Qs0U` zaCe}@8)mC~8{M6^A53>S!CKV;ez+s?cz^j+&Ibr_h3{gWpI%KLC*T2m0(gb_S`h99 zMN}>p6UaETzv3ux5BC)qEgx+j#p&{_a#WO-w>HHiKjOfKH0-Ds&Y91fe36om9{>@) z?CvPPGEj*{MV3tBFf%jTK{J(?@r45Yw-FT? zKcYAU+*9yWJW|Yq9g->Csi2?1?Q^#Ybrom|+_EeClaLy=CmayqX60{RcW`Vj%I$q{$5{> zPNckFHHm3Avx?!D`Eukgq} z5#>w#vqD}u5<=8vXRAfXQ?*YNo6N(=uZ3lgOX1RYBcgqYAen+xcDpRZu#}FEw_Vx{ zkC0>LMyTJXLZLIGNEic~avrN{c<>fHMJz}k*4vhaJzj-HWrYwBb}SC-qMp19J9`I! zbAvq}hOLD1?x9(aPYYnTYNe#2IOQKD}Uyi37b zDDM@8_R~M)pmv2G)jgM_s)a^aMzdpO%FJ8OjLLbIil>p*GcPcY7zTm0@SQ3QH(Kw! zNzf9C|7A)U5I z%Mslb*QCjhvYKEc(+3bLZ)3r&zkCvFynJsHDax4`U`$1OQJur31Wdt&r5~5dL|J`$ zL?HQ9zza`Ra}wLk zBl-t^doP&n5(#+{G~3J$%zX)=cN!}WIWpdg!zG8J;`J{ehtNJg9l3icSn2&)FSs+D z0)A@Qd5lYeFiF#79b!i92T?`dfQa=q&YoxarPxgHL?XtngwKB#awb@e-hXQ?>N;}% z>;d+AnMp4v*?%+$5}{obZ-M=x0ytEY5>H@5lO!eQUJZ&9u!zIFRV}F~Pb@>aAFrfy zH0hL)k)DAeN&Y_IP$`KZI7uA27=bA-K-aA;IqF~k3o2+m8jkm5>Ww4xTk3PDn9)|q zWG1nB%rfVkd`$h0xxl=^24M$)6XGZZ$Cc68wc;zTaPJP-M(x7c2|)$1Ur9=yAkglX z^=OI6Ikl&UJ-Pm=E12-^*s`8V$OzqJR@+Y6< z!arnZ>Ca=F{F?7@dq@thUT1&xmiSMcuLToxxxQJC)7mU8hC*X_E3HTwsblUz8Y8@CCM^Ui#OaZ|BcUe6-r|IP4&20aiPZnSNXIXod-ZDSfZ zWI>45=IU)}q+M5{T#!?FNA|c?%qM$%Pp+Jul5$|+{IsX2eNpF9-b3~VOX^0TjK&KV zvWAg*h$oPq_1%ta7FrfW^lH{BT$tW~sI%*H+m^|mDLW?~PRi=UP+BR90fG-(UH!?b zPJfy`Vh#!G6A|$rr!o*+wiRh`v~I}E@ozJk9Nu?txuuOM=@yeYQ#ULEnJ= zXR{`yX}5ebL6od)^@5SWVc`6YS9+c_VnWr!|-3RQl zJX<&|r&7=JfD`bw}!WB7w+L>%POgF64Nnk(m#l*eO7;Yr>&k&fvM zf8dATdS$(AFrpm6m`BWiu#arF$-D&Zd3Fy&`zVal`-CDLQPY^RftJBK;R!3ic5sDi z!qOt?y(r0Bu-!A5A_q5}=2Y^$O@0z(PX+udc7Lu<&g-X2eh>L`pnT9Q_F+WH8jt&n* zkuvV@IQyHZ*>Mu?I|qUDk|SS=&m_obnkBw6MpXouqO);Iu}e|v084$NB!-dE_Q+Ah z)*)7v6S9y@`csruY3_k|;ow-?Mh%Mm-V5xzeUg1se9KtFSWrl1VL)}9?!_fY#OG2YkDFV+BpD%YMi{A~5EJ;Bn2>JL*Gq#QWF<(aB_hCVkR`Tk>OV#Pw81mz{-R3^7BT`Mv zRKz^?xVJ}va54osOGN1Y0d44hBpW#Z6!mV8@l@M$wu|=n?RP+z;E~?`QT@495ljkC zSc5|lRlkbtE8PV@aRj(+?+QG9n;tcutBXiP+J#n-wqSDYTY)8-c;hD)Fl$I#Fc*#Y zY%UeC-|?{ktPdY59)`_oXTy_j6>Te4mkOZF^QQ>rP{G2<2%CO+o@mQNftQKWQ!fH0lmfRsB^TDb^rOmxvC%8$y z`6IiG&9@ z-u$Io6Sy*39mc^Kb8&Rps#Y=6PQ}74%NN*mI-?O>662;1*UOQ{4T$%IZNK@m`0ca@ z=^`X!JtQ0t66tTJS5Dc%XM|r5B;r4!pJUU1W>+8fH6&44;ltB>Wb8BD8=Gc#sxxx$b$=iG5neK1MsakXj8Y`N-waVMt#ELu80C>RCFW+YVj`!J30=M{<~hsv~U_25`joAt)u5U}*4z=3 zrJI+vHXPX?KKc%QVB?~04!y8DanXwyAQa?KId)@m;4u1lg>2m9-4pqk&hQ6Du7`o^ zIbZ&2Q*Nl%hPkNj6C>febA}HekYj2f8>MmmXG5t^34OAc_f}coH zreI7L^AXxBAJ~OrK^Bw?7_~}4mz~e%&tE@OjnUV1!LY@Ba?5O*1v2s-f+|U54~^8r z4+0m?_5(+Dg1cFH2uot8g4v3rYSJv2-gRmw&kH%deF3<+3!d1;nnFjqjY_}~_>s+w zH-bEo4+h?LoxV-a;Rs6AiWK|RrBSk_D>w||DqpkSfdY%lE3)Ja*4)TJaX_fiD8<`X z{>V+!ur6|)p#W(&h8AaqaOc<=FgP$n9E zTd2}?eD~$-Bb1VM6zrOiPs;Df3&e%uA~Bb6LOx_{L$&JHVUmAP=sC##7_pFN$04M- znpUxlL-!OVgrKo~0B|8Wgep!?Wx%7q4u%S0EbD1d8?XY$ME?_==w9RP}TH#0*O`IVq1$F%gU z91)8D=R2YkdKD4STXzPuOMrTKE%)_W-*&6UU+2^Wgsa)Yo8V_ayLS+%0I)o~wA&B&l2;=>8!} zI4hQ#8-tS-p?L5$ogKXPJ7Dj1srA6zzI&}V3cGy%_SB~x`|5T$mcKXrQlXMsl;3hm z%syl(9stgagm0v&`PV_AfjZy0^|qfP&Hezv(uC|6WrHd-I2Mo=tEu{C(A>RLL~$hm zI7ea8m>DEWK41DsGF;6>aM~z(bd_9 zdl99LBNk1`WdM3Wg}+lDEhv8mD@UJAmARdgMrzgjAY0_PaP`+FOz+sfMAM&h-3`q&eeCjBWMx?I&5}pqyE&DIams{2_ zmEjNEabHCxD#L#BY=%9u3GX#Sev>V(t;q?&=)QW?q6R_zHwW_SzlmJ)C~!A7%Q99m z2Q-6<;ZA)qj`U&NO*3iu)U%qfw$zqYR6Xm6r9x)%H;eBlb8p7Ny^}Y3WeaJHaqG|; zo`-F$eVqMSd-jH()4^Cq=lu~3Vte-j`x!581O@I~Z^N{yy1yahL0&9aUS*J|K2p0b zhL;nNAb65kgoQ?g6e2ze0;shPO}-+zL!XPIli3pBnH&$pnLJtw;hD#+7&i^x#+Rpy z55oy6P-DD)CywoSGagp`pazeHZI5lr8x!2!O;0^_j zO;ux_9G=V_Q)9>pAXuN9k`!LwdX)~IjGn8X^V@NzCOxbpD3#(M^e zx4;3Y`DI1VEs=7i!YNq5v|e(&dA%D;%d4vq!x-xTa58+MSxT5o#bAtvQV2r^U%0(k z**K2uNM?}1Li>?iHPQ_{0^BRzh5R9(%!-my@B9xuuq=tOcbAAd7h~RmIIg|GF2kO4 zSjtxdS0v0v^9F2_zY7j+sT#bRECQjn)70saDQz(ax~#WXzGPh(Sam}|K;u)}Yh6XZ zLkc*&eaP(JRj(Aik9HM!P>-q~rWdqpQKZ&sH*{YBnjJHZ&{BDnRYr*^jUR05p8($5kgFnx zr{ZT~^FBAR{3mrq^&Yf~xF^l#?RVa_T)ZT6sS`{jJ8e%0%UlLiSWFsgkEA#3{T^99 z!cxlJXRE)}Q{Vi6WLE?qCna2QH>=DUiPql%2aGleR?Al<73or1uPnjQ-ag++mr;BJ z8+kf2i=2sv1O+e1F3L%ntVF~y4hbri-TLit*;E5c_Bd%p66bzU6TVv9!8*xkF{?DA zTj^n<@vI({Cd7M;PGk3Lb&raLAnr`zpuG)&mQibELN(hgbZ0% z9fD=JgTVQUSSeGp%o}~($rgQzU0?2%lePllz$Jf4fElEi$Ya#V!D~p zUaj)M7^2tb_-aE0N`^kk?d{NR#hnY2miu0i@a69|CcuYogX1whHeqs3|c! ze}r91ZiEz>UO^_&JO*Bh?;4_$*0BqKLG@CgO?frn9ICW2k)Tfzdn z9(f6hQ}k%pWHA}dFHo)qxZ{%vt}B||THF*MU)hcK6w%*fZ)!hKOel;vGx$oL7kl)i z>z&xScNn;FC_3mX>7{ZoZM^;cx_SSQ8&DX84j3^3}SnWc5wFMU!O< zDHD5vBV}V2KPkn^nD=2UdK~O}Di%(f^NDjMI4i)>ws&)LKV0Ck@JPRNn=}^7V$XL4-l$? z;shK;9N5LUb5aMpt3wwt?Lbx(HgHJ0Q&Rnz5GT5g`ZANv>|y8hd*G0OY#OgX!qpzz z%le#Y#6RFIuzyDK<*$6zy5Cf1VcP%qyt^RJ8O=K_83mAcot}lPrK7ByBNkpGXjINi z>XP*Tgr3S#(r8Xgqh-J^Ih3|R-;R*m%aZtfQpO&HU%j7~WlFStw<>@7b%@Ut7}od) zt*(NFFnnbAW&zeFv5u$T*Rd#g#zlzV8lQj-BzJxYUxtNiJ`zVxT>dT+OUMlIZZtF$PW{t=>mWMS;MwDZx%SCH!-0dCwD z6DkN{=8zNsTsz#*vl2peDhG%<6%8?FM}TXaYot4$Vdvy69TvJc7%xWKIRdtoS0mJt zi?<%L3a(ogxQ;2VW_pQLnt91;Usei?4^9AYz4J9mm*k1qm`9p)Y7!|$NE9tc3X%d& zP3w~MJ5OXttK;;xNvuUxf4LA>6Wgz?s{D9NscU8K z6)nO3%>BS|&0P9tRcGJFM1Kmi>V3fBnZyL~OlC5Gu3%jvwqR_6IHU93lKINPe?YNV zLVi=G-6lR2henqN_pY#HveyjJpIJu=7P(U{Vo52>kqG{FH?WJh;rf#O@7iP*K<4&A z;G8Yy(;IFkR&tB{5qLD8U$*!wZaF;qg;_f#_eKR2((f>Vd_iJr3n!! z>(%)XEcyA*q(TJhmGM~5GV>F51${=9;s|8~MqQ!4!XCMwdj(GPc*_E&GW>zhXfJRW zkeO^91oI0zhNVB!BE=c7FGrFq$&=_Y%f#8uyR}{qH?xW%YyH$to ziWMb_t46*VnMeIn6RcMtyl@AYxv?0yq`kVBk_-e1KZs_~!UPcU&FubIt{)8GVx~OCpNw0i32^cx0m(1l%y#}xI z**e9yOOEKS>OO!A+!xr?NY0lPg9V6UL77+$f4>k8H9;C}bkXUN8)DI0AlBSU3fj54V=?fUIl zjl2O@Z4g7C>Cgk$46?<$l`GXG?=8?{^+VaMu#RLQ9cl^NlYMFG?=V??9r)eF%yu%9 zw4yBH?#Ybh7wd2Ad~OS^CFn&}9)|T+KfQ$`FP0w@-nvKsvhfV^==bpsYj=)L-DCkz zzh{x-ru_5fKv7sbG#uM9QZ_8xMGiV@@LP9g%G}_D2oQjRlULpyR`u1P9n8jdM7syr zCCIq0iCRjkVU+GDyv$Noe>UW%r0mwcnq6>r^yg_Bfm;`dg3wzR2&DZVAQ;Bo{?XOG z#or<@RUY&^=Akn z&(yc=r!UbPm~{;L&a(cL!&WN>?^sfLR7e>&w}L3$=e-3TJt_A0hn+=%=<5@q^sUuaNbyilz2KYMO7UqkjXzU!Dd2l{c<$GV@; zSay=S1`){d1GY+AW4vt`B>P51cDxjc%J-Pqs)4Y}m~yn_IVuFX7uX>v{jED*{(DYC z{AG6HGF$3*N;x;M&%@JZ$`wdW)<3dm zofTm_s)Z8KOcsNR4m&(C?R65vu6J8Bdl1<4ap1wDR?j9W`61$v3yrCWO7R{OWIrE$ z`;@gJImj(O6_MW;CxRKSna|_PYvl85FSn%UjGLWN;ccxpnW#Qj6ene$jt?NB9dHPDX_Q6nF!^Kg@?DzQum!7YrNHrZ>z` z%0jAl_XO$^#fWr5o-eWLqX74*cu`LTbBg_s2ax9bHhMLADZr({Fp>To^&$IA1khU@ zL$~pv{POci0%R>`x_N&FZ!!Gg?0|@zy76f;Wa}mM8Ee8KUmqPUWfvUvp&@whcYq!5 zCF%^dM4f@(6|hC}`*;9VDmRiJyfRCFg}^Ml0ddu7C#RAJs^Wa*gpkci&DzSHQ!*NQSq%183#`~K6I@rsy2QCC$kL~-t+!S0!Lph=klymy z(jt}Bd$m)3=swL&c*sn>jJm|w!wD~-v|ddt4Mx0&(0~NS^%EoN8!QBfkthzpaJd6I z^T~sQgJrswmKG7a?eJH=z~v$F^4b*YP<%)zav`&Sbt!480!KnXz7o{MV+5rS7e6|VuG0!f^^rD*q>>Ke}E>GCkNiHHjImM;Wiw;FUwVi=6R4*Z15H1NXD z)Ek(}@CU7Bxo+Q%c`0aFw~Xo6^nXPVS)aw?vVH_r;Y_SK}S9{cWaJr?YE~kFL z=B~J2^u+0@(>BoT?t(n{dGm4jLw5t4VQA)!hlgAfB2R4Sz%YFz{Lc zf^+9j@MfKE^;UlwnU}u6W?qz`c;Z4U^1+a!f{dWRbTAF6WbQyb?y;qqe$Q4HuJnc7 zMKhc3xP5mS4N_lv1MJ@R@v|g9F3;D7k-^&MrE-`y1ppT`@IfdjJtZA@e3JfML|?_d z%TY)sdxWaR^oh(-lzG3+PcSdKk|ngPSvBqgu_gxvjFpKM$xG(tPZ1s%Kl1>8c)8FE z*)4nksbNJr6Gi3=ZNZ&WyRu`R59Oyp_Tw6kk~Wy{qkN?9d)CXp6ReRJE8RLPK4*0$+i~(#CB@mK>EmjiXHr zo}E-hR8R?@aJp3%bm7wMm2J!KaQI*xayLi2jY#`rUVDxm+RadOd6@T~rl!w}+YEm= z1^FbBw+V1(e%Iy6@)aAH<(>h1*_wc*l5c3jV;oJeCN=&3~YB7!kye><5loJ9Fa%y@V=<#tFD#?V3bj@D`t_eB0_`8`wuH z6)wPp4B!{8#1u=Ai0LHoOd~9icEXzjF{R-TTyFlr>6T){&a599O1Td!88N~Xa|uP? z7rf~BxjWc&X;v8QLHGkjgoAlsFb|KUg4~h)#vdrM#oGcf=(wlY?G>GzBST=RY*3aY zu3l8++HElQJ!zRs1ZHkv z$@z1)Q19rImc0f++`Q;!NZdTSBUy8UPahDcXLN>%Vd3ATV>ve*AsR35(I6miKVWlj z7n%z*lt!%4(0(YL(daB-+K$Sx51)ZLw=RZNBOt{@YWLH9*?N?@cL=x|m$>P5+wrN+ z=L8Z3djM(S4z{+9DixP%^n2L*^9KsG+#C(PMrC8|W%_pzT#T?|8~M0K-qM}52L_kN zfX55>N3Pd@5&bgjCCm&@ z0&m{g0`I7pc43NX7j*mPEe}k}bQO#KNQwHWWl`5$W*;5Lg38|)(sb21v<21M!I^gm zxUJdpYr2w|)c>uj^-%Z38xTcCKRfwD^+OT$O!*A{_9NR#fwKz_ICBBT_)!= z%9rr^EnY~!;&!)cX~d2_jgEp{rpXC1gI1LbBI*+LDFTESmYTZhlsP#=!uazraH}S2 z2xRMm7k;T+mrZBVe`g=Se&FMjKS@o0?+_tUvf`jT7L^JA~TAY0i)<4r-w1BUl z&uxa@2iA7n;-^^KM1K{IBv95UFlmw1;LtJ7pVIt?OJbm)!PmzP<5O6`Y9gx|((y}7bZ$QJDt!mMt6Ku>+L}Q7UN6w=? z5+492T%c*fgmP5eR80Q?RoLKL%PB{T8&i&QQU*mnVO~L`qA3tahqImeEy=eI>9jpN zyW~}>Dx6ADreG+yqIic}5vB(-F-@ParMX zdE}SM#zop-T{UcV?gRF-o+dkGm?{|?pH>D!fW&TKljMyKQKp?4u3`4FQ^^xZ%XTi8JMmPMLj_%cY#v_|wg*{4{Xiea zxUe_l4L~_0TasPOM|3{z&(1S%=;Z+hJF$&*r50VU0);V?(!ta4332Jt^>qna7~?Mo z0ChQbmu^-eFiO6ni$%eG%ApUCU)cL@sEtmY)Zk}+nE~pZD5+mS9Y;W+18MN zm?Jz&WSI;l=WbZMnI)l@sGrY%h+I!_WT#}D8Hbfj8^uZB7ca5G6qRl1uOOr?p`v`# zB6e@OreA_I8g?Yh$0R&SoAmAM=3gMew;_M>Kv^#F9g2v)Ttrl853pH9N5~)5j$m;k zk7|+5Ljy)7T7_E6bBT#Bxjo zu(E=&kP;p8bPows-{d!f+wg0(*QP`;LO23kY1ay0K2gG9^+oLF_1uRTr?eSSWi41fOHRpwu{l5*odn0BD%*PwID!Bru=+lV{-`w>%IpLS#OHLrrO5j|Ajrf z`T`NYI1PN)tUd9_i%0Gc_FhGtUiX*@e}i@9w2aQA{+E51Z-Tw^_+vavwz#9=Ab^yw z(As7;J+2hPzU$7G`5i>d686>IuMPMi#0y^lE+G(wrjVuG?ZH_YVx%4`OrItTJFvi1 zr5sKW0TO+2-T>ZW_=7^s9n;OF{${X&8$Uzqb?&VqBjdm{%BF2PdQ`~IJ}ultg<1Nb zvQ)&>Q2&lHP-CQ8U!y$tpt4DvR!ILl8%&$kw5a=k&DP&;N7M`~sb^G%<8gT-zlA@E zFF;5On3`G4HId#EK$Vh_|Gg4AsgWUd#eF~lAsg(nl$E$p>i1#k!;1fQ@HPBa$zT=^ zbN!RR^A0wK8FwJlpC+%VxI@(Dj64BwBecZm&|JfMV{eTT2!z2C$SPSlx~U}V0pM6{ zPD>g|o09bQk6j>PjmDDNSKqv-DtawOVwm;poc9M#19pRUrq?E09@tP}6v3WzuL1Av zAV`St1U61&tkiwo<*we3-GEM7!j;-+QlE$;@7*0-_&q`p7jA$(IPa5PnX!z)0`Qw) zUnqG;G$yh_x&SRDXR*2>`VBNAFq?Q0vh$$@jZi80<88MA{w^7%b_@=#t-@h=Drg(I z5{ivxMzH-XPa#ra!W2Fm`smMa4GVg9>MA8y(l>tg3g5jPBYxWZ@C;0(Ebszv+g_gz ztH%(op9O!*^j*(yPK05|{C({eY|3Qci`Q(hDkYSku!R6wRC!2!bK(Y4!X+MCM z%9j|mN&Ouy`3d#G8~$g$&u}_X7v$E^O3) ztF13PkUY_>l_4ldGq?0KP+L{}@7&&+5yx=V4QlHRS%A~?s`rE=b zjuy>+UI)Acr?ir5pCX{jeZam>S}$#oHcFeMJJUGti&5cCJvdT}K4EcB3`@EpS*g3N zEMX_wM~2Bf>T<`WPEs#vkTgmZfL__1S>ChZ086oN3)0WHo3!oxi?s93gIyQ8XP~<4 zY2cG5vdA`-Kd7z4a+uZ!7<#<9o8nktfpDty$Le$HP=q~r2)J-uXAq&ipgzy`dAfn) z=f=~K+HhJ~o%##XU9lpf1D@oGAKoZwmLUm;fP$YUNF!CM)GC-79tLiP#Eb8S$>0Ty ziq<1ZvK3PiIisNzR69lE=r7~}mbS?XORB!X{zAZCn>{dEI>^>$sUH}k)@Vm~cUA8< zja81LxIdHP?b009|8xlvq&H-E9%@&OtAK2ulqe3|gDi(rR-H1)%_!HUyW=NOpcZO! zK|c8SmQph&=JtaTr6EwL#;@gsW+N3o6L;K454=Ve+aCak*z_~ei6Q4J>~CjXH69_1 zM6u6InMeL|;4&8dqGS_c*?9oz`5mi?;$D7IRgv;@%zW}+2_&0%$b=}KIi?spZAshetM%m44H!t z0F+5yUyQLii`V0%tL&mM@>|p}a{zqGY)5$P+lIwFp086My-3#iJY|K`9#O2#w$x&{ zE8Pc|^&6p4+h3=&6C>t{{~z`Ew62XDj9`@;g~eQWj&K9!!)7C9qh@1fG%3M8d8@O+ zSDFnEWrpee(x=$O&SU}gZhs+uA2_Mk%K%I#`boGGe4>>gA!Wb;fO1K+jrxUW5ekg( zB%sUHMZmA1+>ajkdn#Nw6{)Q?x3+PTklA7^}l*HPdyXNrGwp|JitQAPE>T(lTL^Fm;6{q}a!@ezVKT++IdFlTletLjaOG=_nc zqXt1&;QN#vLk|5d47@d4fPQHfizp|4fui(j*6^Dr5WgRGR~aKrV)_MyXH3vcEjll-Tx< zsn)}UqdN3^po?Qj|FwAWU?adJcRDtIQ@q@HGAD4jM)9Q*_0ugLb+U=5aWAztgX%Q6 zoUzxldG>}ke=oXyItwzZU70pd!LIdAz=%6~BGFl5oCFfmcE5`p7hD;`0?;Ro##P!S>5m%V-AFx<~FUx#GMM&_dOI}bT zZZMPv4z4=H=E9q65Pv#m-+o;6DhYCAT~E1f3@>zxQy9PT2@-!_Vkoi~)Dxoma;cO0 z)8kB$e1zS=OoScs9xnrv7z<67!^=o)|7bcJDGhXC5ZTxwFUn@6_u=~C2t<2gl%;SW z-C=Fis{TyP)+@k5iiixugrvsJ%{mt;BDdH*WP)KzZ6rz^vXzE8{d%`DMid@e%bHu0Wh-t)*lq zh$oDRc)}Pi`G$4;Cd=Nf*+}`>o1bi)=4IdL3u%B~#Juwq(GibmTU-GaAp(Y`0y6h& ziXQSkD!!8d(L+h573^pv@dK5N5F+&g)@LaLhD=!?X{_*8@Q^UTEKOUVcm4ey*=KM! z_!_vUwEF*0@G?@$m6I|@$VR@zq-H1ZAm0yDU?Quumf;h*-meTM-|6<=YlGX}l6=@?l+l`Y<3_M{!T9QR}{ z63rP%fbiHJDZ|MpDzITsl@+qszNr}8wd}WrMqngCU_>yLpLaZ&iTZJGezj3y{gNp{ zNn(1({dnp?FF-oTsA|D8os-D`^7WZUh;dT*Cd_#Jm9CP;#_;c!RBm_iV2aTVn4gji zHLO;I(&12E7D?t}NJW!Q8#;f_R4gFcw;88}<$D_Oy)^I+5v`IUVfnjn;Nfh(Zg`kK z$Vu8*`yo;%VF-CVc<~qCorNQs{fi|{iVWhUg##GrO;LGIvlDg}=+Puo0l;gPf7-qo!g_K6rf2af+UnS4Xna8klK@WILImxs{1hei z$nT$qHTDSD4f2$Em|{@zchu7Bsw1h*<@p|dij;@xtPmB%i6ZtoLou2%0K?1{dds|V z(p~9}Ar-gGLg>7&Rd#Igk#BtsSZP3R)ZMkHXf6z^08c!+vw$I)cL@;mn^HXsZS=tW z(&RqKz`doXh6^kQV(}pdd#G&3Jz#(4tfDIoD0&iFnToB|=Jvq@vX1y6n6QfWQlLsc zI`UqE%Bh>~&+GqzmI=|BV)bw4N@oZ$jIs4)A(b(hSLos#55cgn>2A(%uu7OMG=cz? zK41B`Om;)XXg5+&eGza^PLexzV&Qu20Q<20`7**PyAG^(*gBpmk`M&PD}-ocT{*&> zq06lYIYX%RvFko_w0IWP z{0^{VJOrqJA@%igi&jig`nEcJU4pUWr5#FGllX$z=8f#SV0x4r`D*IFX@EoxKx-Xx zi9D>xle9!+7K+aJrztI*hl8!3g8}_@VCBQ|Wij!}$(}0fpsbRokXL>Y@XPpWns$dY zk6RQ&{xeE&WSjMw+A$ zowAyoU>&-W{4Pub+%t4$6i%eY;V*}BI}zuDOInP6VN6pe?tbzIKJ;Z^wrj5S;k7dV zSAYcy^&o2c#6NDdkoTzp?nuYy4?^*^^zuU8<_LnYOgw0Q_EX4N)6#1XYz^cmd#8D` z(V79^n+x8sQuQ2A}s`iM{LH$c^VrIs)xcI7xme~mN+jZME?sy=)DBY z3fWqC-nOfJM}e^44cLi+*0?q-I$`S1)Y!EH+?qE~%3OW~n~0RQ!dZF`ICzOf;}_4g zyczuvM!iRQ(lz%r7Lq);P-_co#WVyJ1w8|I6@i2~Qix_1Y`0{@VBsP3P4UE(Cx6fr zwr=+g!B`ql1rY)YLU946i{d&yy^#+gDsm^;=t@L)@#>fMW5hCr%RkV&%&$}#$qQ!Q zgUjRx@YrDw_>ZW)ak+Ao)w@-n1uKONFA2Ur z`1s2iE4T}}0gG GhI52%AWoC>tt2C3qNMnRe*&JD35z4y@C;r}&AMayBR?8nIw6 z(gv;Bqxot9h2jS-n;T^~Odx&(ygzO!6ELXQ8*;e<;qk({eBW-w*4h=AMe@G#)vrwm z&;w_P=de-q0oe*=u$Yg|n#W>h*~r?jdOoL|6$rqrCzw%$WyYnM!wJY>RV)}T!Uo(U zZ@}eVhXN%(@RP%y@b4ltQ${nbfp-o92bxg`gBd!J1^icb$e5A~LDjR8J(?9ZMT}a$ znNm}a`08F^wCEMY-sz87BrzI1VM^s#sH+v<&*><>kHpp&0e9H$j9`^t;DbpADr#8} zOg<;WDthwr6;}50MktRc7r9a2$pE zjU7**)qWBn>J{~gTG?&vZqdoJYA%CJ7Tb~EJf4aCCh`#(yR{+MN*}gCcFT)^|Cq{b zH%Q~LjXm98%9TX%knFs8r{S6#b z4nR;Rp{^GsE5~{|ro1ugAjl-AWt!dh06>_hQ-jZ72IIq%&{1D$lALfPqy!sD zDOu4=;US3MoSTnFN01*Nh480=W3RA`qiv(@W2T}g|0IffV|+VI%>H4YQ!o24U<&x_ci!~hYxexP$q-e##D(^I^6MOTRtnON4qd)QtdbSXC&9)42)aFv^HRum}bd? z6d$8xQ#BcS+m6axXo)=Ohz*yWBP(0#P+rX~y}K!>l}2N#ZQ>_XaNq_kp0l=$wrzH7 zc5RTfB1n)XU}QJ>ZEBL;2d?@Rd8ir6FaBMXy8?rxh|ix((Q)+Y?|sTM*O0uC{smc? z($9IFw6^-s5NXw2s4^F8=QWa7%0iQ0l6by>!nldUYw3d>g$POQF0fOX>sj*02eT5n zt@Vn+T-29HAG=&N+ryb{e|a=OS4>u-p_?$FaS>l!f8NHX?IW)k_Yn=kWccn8?wMK= z&w;S#bzpgBQ%0f65_PU5cBTs3d)?XROf8CfREH00*}lh*lW=q@Gm^HRE`HN9-Xj=+ zZ765{H`1$21}k}1ISF~Fgq$I@VDiZXYrQHvhK&ZTj1DWqtxJ@S68X84ouSDNUZ=ui z)NHX?5eK6howT($N%&~;uMn6_2W_mAJs;^o3X$ObIB9zd%P>Y1}y%H@$<(N%vNE+Gfn>1NKW3I=BT2+=rfn~ z7Yi5EU;Z=;b0!lRF4Tf#K-qKS$5CBlh+3os08b03Jk(v^jVm&fGhvGIGB9o9E}GE& zMFwO8IGLl&!B7INmq=8>s$?-hK_EmKf~i_nkoI~B7%MN(5%}cJP)ypn(uPAs{3@_O z;ar34X<8(y=l3xOO}6fF)h+cG+}fE0sGQ@ zj5wkSa35$_elA6FLyKPoa^PY3Q}C{aqv08~+8`1n4Y3|rB3p%UYBTvaNdL<-KZ^pr zPF;1DM9awG*Ml%62?EZe3xGRs*$+-dCuW3;2?`@PRhhAsrJ`^6galLw;1aB_=xmF7 zkT-0BfVr+rCEUVtSvgL|Dq}H(`i|tKK}q|cidqt;kaOHEJ!RWKo`UVpcgNvKTKc-4 z_WZ*CJLr{h6OskofYlOrnY+SW<<7I|0du;TqypwMGj~L(8RSZHF>-+XKJ^%OfZrgY z+(cvlmQWJ!BC-e?yyn?~aV#usXp4&iF`B$^;gDAa7MH%;X$$vRXAQSj;Z(*j7Fyp` z#6n3Q=8Y0PJG_nH5k@0 zN9YzlGB23VxW{-*vg=&-fk&y^>N{!>y#%AuUr9%$zk=NG$%WI}fTXo;p)dyDH?r?6a~Xo3Fq_jE-n|{Mzfo&H)UeY#YFy5`11>>IqYJ z!)5t=k!)MiA9^xg&msb7=li{~_fN$=6X^2e0!)IrO%fW>wv6z{R(%u~kmDo5jPit)&yQ_AwF|RyX5Lv$h%%0fv*k~YvRn=17<7d5_Jh)MLQ@C|fdrCX96MY_B_JOGlJs65fVeyLq ztYR&2_y@7~vfj7>e}{4AgNSF(DBzF0shcwjq7Jze^1@vsMC6uPQ{CMvX}7@+ncJ?X z=iC{#<~i5nfg+r8j3!Tx?MR7tGtGv##G*a}IwAF8(+lny#IdAi=a7HDlA1X_%r<@jdEuyXoML`q%tV z`DnrjEmdNrA%;jb?je$~;BW@%*Y!4 z4IV9Bfa$XJGP{Z<8|Fc>6*2{m23yUNKccT8f)BAI8*SLqiB+P!&CVCKtd-9GMwJMvQyITPY0(rhV#&)ANnOBF}9bV zJ(6mMECafknttC?zN(I8AtTO}ajW8wnC?}#{*sn?(xLIZVpq&LobRhDF9&Apr){YMb%f zor@_?{=neL5dc4X*kWbo6;^-p2e!BYtWVCXgsHYFxUeDE|4T*k`eZ(OBx6VbLR4@1 z>3{n+Wo}|9dDBuVVWb-?ruVR}J->2Oi%dwv-?d$ zP3`O;YskGVpNC9u{mS5FE8K|CwNYd{y`dX8dsc~UtkkXJ$kla=gL|_5#-u-`n(DOI z3MuozTZX;ok*YLiFlt)Knw-dL0?1Ikp+}L6pN)AhZ<-)0&Gl24-8By6=};B(f!xxE zm8m8)oJ&_@c|0Dv7gEJW-hzZKi?F(br4FS9Q7`qc^{aiw&0x{_4dyUBasnd%3n=qe zmf{#8mrD_mzROInNDw*7%9!=M438ELfbYe6V_z&(`o^+zLb?=4+S^H+0vbs075)T| zn&eJEG)~hpeeHS7+Nw=EhPC?-&Lx5@j92C2YX$xb2vyJp7`I58#VBYd?~7x%rKk4d z2s402^*uN?S`9qjqhMS+@g-nQ#8NGUa99L2ZwE7=I-J7$CXDIY>=l>~rx34D_N&?=GFaX^lM8swB% zG+nD%eZgn7*#6AWZMcD)b2UgvX<-vD#}0WgR*}#=v;~b{4}8(Tj0zt)(iL9f<=D%( zv>a5+`zElRu$r`DmedEH(n$7juapu|@Lx-~*&x11F^(l~^pRmq)isI)KLU$TgO={|-){H8${H2c||n~W9ZlOH4Z zwSQ%+PZ(MlwzbCVuWOqY!Nt|^BH~BRVMGd9#DOu`x%6DXMBJ}Z2I}RAo(#jP61Y8u zMb->^^UyF`LusUpFp@437hsTk37D23m>vpn~lUAQI{7WfRaO7=tjk;La!U{PVs#4c=K z31yEY&UC;{%$DDM>g?LQl*sq4_bV}i-|36sme2W6Cc)#k}|`09X+4|7GX44gyo$QvIVFOTHXjy1X&46)?MKUa+SH5YCr|EE4RS~hDC#FA60uJV!J6w2v=~uS)kni$jpwW zLd#J3(Foo`{qsm&?5nW=M zUDcN^LO(M64$(A3bziU*yW)%_@b$C}GaK94xQ3i<&X`LTER6zxfhUO*o+M5C+{DKr z$&6;9I3H{rmax)ETCaaGKf!(GGhNEP*!`dAw6-n_9SZV}*O|?6S#) zTrrIxrbBUjs%(Pm-sbJaLCYrf`sr;sm(_(w;Yfga61_06j=%x+&?8 z8S8NaNDAxZ_-lvU54dN4yC5|-hzk$d=L?lQ6<701$#_KlRcC+0y0G1qDgvhoLYOwQ zTiC7aHg-D&>B`&)Uj+gVTm=^8tOHDY2=7HVera}a_K^rFDL3`eYE~DO^qu{o+OWIS z^t`2i8u1&A9u+~&@RNYZd|aTtJBkzvnG}e;>b z7K*@vT&MVvk_a}U18@l#Bc6<>J(y0Gdt-`tk4;m-FTS!8zK1U9E^I7WDRslNPy zs=>n-e&}1kR%^|K?T;RoTS%XI_VbV`3q9=xeqQ|0hO|5C-f&OWSr7hM8DJ&T4Z?$a z9*{DbFASjcf~@?>>p6k1?joy%4y?J=yfCpb9+k|v1WeD))F|1t)%MW)(f>xaOJ*J3 zCM=<(6}25S4DE)FmhQgmzfc^u21^z{1$PaUvN1;Ciii+SLq$(J8il_H9BXj{ZXcj6 z`Qnq2gJM3Me>zZ-g~n3c{p1g=l1Q6+7lc2LNrFIySnNO-h7HZarR@p393-c3UmM3|Q%pv-rWTTF6kx`7-FVzg;G-3)S*?W51DX%!CfJ=Q52LS3=Buj0^k|FKH z$!y|psd@T^bBQ-MmKC#i{X2G$v1b@!RQM_Ig5dZb3ZTc|qC7p<9cxV4`3B6Rp-yyT zPq6;ru_vhbxILl-4T8rXEN9IpWAKLO3)IKEu)@CdSj2?xJ;NCbP!@t6`1dxkk!`_^ z8M+6@Qr7bEF-beIna^KfNH2JLgq!w|qzr;48dk;elFHI%??K2XG!nV!gmcx zJ0MHNjt69^=rGfPO$I2$j}`8%2qTEGJU`;^Q$2GFFa#C2qU)^u?2Dmg58sE~yb3Iu z_zWH@pdi;m-dY%lL4KMWu;^w#Vej$xHGEur0LNVd^tZy>cNIn?l!oMmBB1z1Q;-pl81vMUs)3l-v(0^_}5-$cTzMK(nf3tI8RSS;DsYI zF<5cLRt7>|zXVJNP0wl`UJH%|r<^uHe3vEJK-j@7jvKJZzE}jb6+ZfiG>i-`MQ_8O!T3KB z3){x}Ocx`RLC}lD9`J)q)pX&IBDbVm4}XC2Ty!V2OBUGE|a5I0KV2dea$k>GX0-Id;R|3Mg{m z42+TU=}8m*m|1*Y+YgW|=rv$Dcmbhd!qawiH2HvR^zetre>Y&d0X9K_)ic{rlQ5Ih zD~gKR*lP;U$WM8@um3<{CT_qS!A#&aGv$d+yBTSr}V(In}$ zXGcCgIAjQ8LMt~YSpM{1^}&_vtJz)w8;IypVD&BIY&MwnEWw2Tpi?TfQm!$4yOLK$%Y zpdWD+9xl1>TbS}A7!Ei9Fn~B%hj4CP>yNB>Wnc6&Hs zQTh44#z^w%j_ys{C}#ODqkH3C6?{=d*00|Dg0EMFHt+W}J5Uet7vg(BlE62h4j)4D zi4c;H5^{CECsDz4DlXklkgHsg_4W?1g|+xvPvJKju`5cO4Uw<0kvXIL7Bf|&*ihd> zjN%G)3v2OhU4jC}#Y2CnNN=8)XWe_9)QM4{mcEZYFt^HCKMsBRE}9h6V)q@zTC4o#qQ=)%@mpaKOt--c3|Vft9U&-pNImpw-G1qvbhz{pSL*YQ>h4sOsA&(<0TH3(R9`a*Z$!`(R< zQ73I1aRyR0&or{?Mv`XwtqO z_-5T&7#}&sYDxEYbO*9>rf0|D5^x_}LiMC{1I@clyBx(~${{2&X{wV7f(bqmOd$Ky zyN0Q%n^pPvT;=K#^6Xmr%Q4b>$~omX;nI3)@I5Yh?Qj<93`U9C%_Hm z!Ro!NaK^7f5Ld9U3H3YA^X%cl93oo*XmsO)i<*ji{P=%qmL>CG{BsGI5q6puQi3Gn zPaE!3fy1d~=A`;0AS|wp$3lzzRY)OYmhI17k zQt#^$5GRR(_=Q~BfZ|CVNsrGUq5Cq^lL_(&Au&uDG!rLVyv3ODxUu$8dRhAQ)I-^S zqO@|u(Q;-fXCqj}g@B?C*D9h51SeCfDrfCtZMB{W6e)`^#jxTB)cQxu_zbT8!;97S zLu8OS0+EX5Iu)DzEiyLwTX3-JJdj1!vU_}&DTUTk&zp5CZDkO!u=lmL^_3+I&hU?b zYd81phHvjkg`lVMc@m$R_d332shq%5AFrB=xj`y1lm<^hx zgA6caCgv~xYfebnyC~h533ThZdR$TUc|^!bH$5|>UYHS-Q7<%{9(i78y|{j~L-o*9 zm&~!CyaJ1ujQs}`0khZuo1j?OWL+vO6T;*}Kn%CgV<_aGkZ6`_n=hib?LOwjv`_E- z1ZjI+-L}Yr)>c>~6UB^sWgt4nNaGSAy>Z5#uq10eybf*b3e5KKw-7JTNoAdmFL2$T z%_%lU>Rp*dp$D%Y*D1eGP-;rdi$ew4HiyIQD4_8=+-f zX8JsWrYNrW=5z6<%qEnuVB9r!-T=Z}qMf^*hPVV~ZR+ZgjrJKmF$-PfuEG?ZsB zB#UzxScr2#b_)HK`D7PsIF*P)i`i}XQ>WEai$yW~;LLg-s21Y_jHV?t+{(LQg0)i? zoebcvY1n~p^yYER=uM=Hmw{Q8ai6tD*}P0zfbAA3zDV8v8nC3PSisZpKw$reJ^xMQZ0IZ!outv7CGq_x})F<`%GxljtmU zY%kj6ZN$~@LrI8>fS+TW1iHmf#9~2>)Hzeu_#^oB-{K@(TvpFk492}gicwvGIf7(C zs#q5Besqw+g1Yw&V58~MjF^fHN|b@h+c2DGVEu)4hKZ(q=6$%-=XV04PK!iys5jXm z>%hr$WjclwGF@a6a)eJ*FZXrMk|!s_bMGx+>nG7^n)J&16;HX}un#GCj6|Y|?w9Ne z|Lp=Ed?bzX{5bOB=!#b`kCvrl_Dkv{?iD>$xKiwZi-13)wlx7&1j!=ZiHj7s&IuR? z8q@8dlXBl7D~Ze01wMbS5@f+(ieSKF+FAY0Sz-qun3Yh}wW6urOX1HS`B_s(%xl21 zid)ZX;Az1Oe)0mxD(RFbeBJK@r%X-@Fu;~7^HSVXloGKl>M_qnRulF-4CNHd8sk_< zA1&w>Y;T@c^g_D!5Q~ov35OhN9k}jDN;UhXXW0-Jeq`NVdPuf zoBX&lE!(Bk5mz@)u4Ng^*-xxTiY2b;66X*|f7|R+0weJO!|J;$Q5{(Bc-ssDkX{3p zdq#|ld93&H#jF>KF1XRTGA+>pkFORVht=Q+v)BN#3nKEl>qY4uG+i?W7WZ!|vzMuZ z8Szz39^i&&K0cV2xr;buY0$h!a!6K9LAo6JVLQ^rOOoQTo!BQ{C=_-VIL1mgEw(IP zSiH1&Wx;4j#kJ`ea6~x9*MN;D1^H^WbvkbSKgcQMEPgIwX(ROo*s-S_uMV4hA{E8h(Qrk!~^h z{T*BgNPGxf=_{J<6?tr!lQ`UT6LLwMfGCLe;tsqIc*>)OBiQLLoomrIKJq@|DPK3d zW)Tk*Z!{LqE!gt-Q5;LQ?Xd`<{PS^SlUjz@Q$z7Fe7N?!3naP-=hV-hdPd)j&L@{3 z-~4>Hed)8$_B#0kVXzOuWw=Y}{qd@|>hr2vVjJ#MI*R_M$ZD8EwbE06zy9~t2YG5m z65{(OOKPn)c9NcHl#$Y~jzP+S1%F;57DT4sSZq+aQeWTE@^VizYDT^_|Sakzn-buUoo*X~axcS6zXe=k;xe87&3ejdE3% zF)fA4P%Es&m;8z@Q7fYH$=lLyCYX0w3(bO+C?sE(bYINHHF@X%LKfrez)GWtiGmZ- zWTArAG({+^al8#KFk~O89|VsC0*^7(7G(1{0S zAVLRyF(Q)^g}x)c53VHgW8TWppV!KghCYD9<}2Xdf@F$6&7a}V@>5u;thBegObd(j zaD+L4Pr&ss(K&uDwIRGdgU9dt;L4M?eSlev@Nt9>@P+f;S>U&U6HzJ-#Q3iOajJtB z2rq0J+QoO56ag4gZmv5+p&(2N1z`%85D5Ug_LNTzP$YjE>*xO*fyM6tThc}+@g+35 z(?gIG+SnG13O<@GO0;qon`2HrrKCMI<3B#%`{4f4J&byruHwr;d?xkcFIqCEatZwS zkG#v;&0sU#%U8{3gPp1OA*Ixr0segT-mAs^3vdUY%Jhvx(bvJrCpWi^72I-N8J zn`QZIz1^;%xox`jaJ{`R>0Q7?vsgg1b-W>FY5 z>xbeNKh276S}~$#y*};xG*Iq4{0VaC*9l*z|7^2yvWrbG>|=Iq>-^i=!jeDJ5*SvI z5@yQbqlDcLrG?&;#uQ!QIzA zMljyw?#CfD15Q!SiTlp}r$mRu>nh?bR3hOh;8u(4RaHRXi9pGPer9R_Xmj z0B>b7QzYS&(LUBAMd>yNCT*GU|0k@9zyXR5 zMV?xbFH0yx6vzpl-os0Bv)w;J9{6kqLj6zbqJ4s3&sJAj6`bY1HBtLVXomGfQ~kN};r9Anpe zO5pdhqt}@BNVFoE(<`QRt9r1h!NAZf)S%OKY*67(%Kg7Yb@Zg6MZG-(V_pzSd|e?U zV0|LfRGTsyUW@q`MCpfV@7}P_ZOKbR#dK}}8;#uynU(p{F}x^O^>@??%6xFp0@ttW zG_nXw!WwdSs(DD`D@^Vf!jiNYy#C}5i~_F#OFBSEUG+)87kbyT9%GP_aue97zqwXi zAe`|CG_UGZ72}2~?|f}CifV&V8ms}qvjE~RepS+0BUplQ8JPNA)Lc2@!-`Crr2gGd zviX)Cr6Q2zjfGYO_2Ed6q_=%sIabv)xPgih>DOp^Jyyx?dp(=YqF`<(V6{jfwEo>vPA1oP_74AZiS4caAEvx zC_xz{{dwW8yp;=kWau6(iND(k^ z9-${%r!SORM6sELdW58$a3T(Wmbz3WOM+z-!JQU=ko&ks!sfTHD6EK%%bh(M4MnM#>^_!cXc4^MF@@g$IX8fi%KB%Qh>9Y{w|L z?gQX_n)tt;p-)su7!{Q&XgxaYKn{pH3I!HDtJzO|yJQGEURS+D4tHTCh=<5uT-A~D zscFRy3fuQFLy6@WJnfOk4-gj6O<+4}sw=w*rlgUWDogomY6;b?Utfu}u1Psa`ut5% zdAUFq7u3`vgaVyIh$64WJv4*e@&fT71rg?uy(zZ%VEbHf#|VWtr>;1gG%Wn zXE##y4+yUn&A|v>`{5&X2Wm4F;yuko@cz+1PX->nz6${hi3Vw%U-Tc(N`^mx4RpxE z$cEclgjAXbJ4>T~s(fy}^6cI~N{Y{yLv555vB*6l1z!FhuxDlrP`|16iDj?%=1d@3 zDCfiCE0bEViQV>B>RxYP;!nEaR^9&Ayk!Ce9QiXM<^;G_rig|7V8*TL;^#^I6&9RV$QutKoMR%py1IbFVlLxtPw7T&MftG zO9#I4Gm1Nb)G{wJde{#AZDZU@$kHMDErM4Tk)Su#KI2G4h}h2KL@K>w5%Q@syy+C z+znqg40k>O*CSa`tZ3F-g6A4E_#3(*ZBU!|VRM&!=U3;JD-~HP5Gi4Wl4E29H&t-}c%}i${EpbOUg$Uu-6p zTTo|!tH4}S74!7O|0H0`8-Lwg^7tL<40DNr+{@2muhvEJklj}|VfT@jNqb^q15K#G z;%t%9L@>fz`g^eEq0yvoBMYV3X-8?km7@GAZ#iDuTB3Z%prFKoQFcg2ChDZ6tL5x_ z6R9KhI1*id09?l3#)Nx@9N?@CE+S~J3ovN5O0k*fIKt;LMkq2%cadElT(jZiGTg zaL|w{(3S!m1+NsWrkb7)VNiDw@HZ$?0R0^52kcOVRjst(=7jVa zxAyce?9qGLJYM=^iW(dhz9H}XVh8JU7&VvmN&YOi;}czr)Y?zn)9ZLsJ&rB+WpwkA zmFKR32U$vOi{fk)&rc*mj-I~4{;2S3JhLTk>!Cd|#Xka9#sqJE>zY!4%pPtMl^{y+}qPbul!h(8A|#fv^_2L^BaI_Gn(K`KaK{`~pk4c?b> zJUvi7BUfPA#(g1~x6q8Mc)8z?raE{>{NQ}|0WZpM-=_~_5&qkGU$GB?ky#`gCRc9h z8AJg=PuPmXgTqN2ru07qS6)h1t;YFpakH^BpbDb!oRb|>GYSLQEthIjBi8^tkhubJ zsh|;S!kRG@8)}{Wt4;U%A49SsbGhE~<$=xR%Lt0d=wR}2K(OJf;8D;A^R~s~=P5flw@x)J%SNWz)hA~zZ z`j%x~?eJIj-J_+(paT!xps z>;A=~u&r;kR;Ek}P%;?qU8AbAOxPKz}vfV7PS-=e%KAgUXy&F+w~;I;LOnLSdiyOiZG zzPRuffPEXb)4C}}g5`I>MWz!T{KWH*=0El=O)Pal1fj%JrrHgId}=T4hD^1<~t=c{TxBY4_ewPcX$XFNdmwsv{)7T&gWQ3)4Ylxh9Ci;1gQwAn#7J9 zdh3yV$7M#?Q_wm7eTBR98we>!V>qo9R7%=GK^?)t7}onVDWknIKD3*ex3(o(#ycOn@fxFjQ0DhBJqu!_TCrJoWKo; zH8%7!tS?#-ifsk^tiDf;WoKYYqY9idrKFAorTjtQROE_erB5GZK9(h4%DV@n;Ojc1 zC5)na4&sffJ)$Zf;|R*gtZy6b$qpH74HCiQsg9OwJ}KC0P)#QVhjLHeWTr&>1LX#g zkBmYH&tWePl8`roVae(B8hefA=2eryD7(@D@U|uDo;@`REf0LM1l1xQh%W)-4X_E? zgp|u1cto<=dOEy6K;!_Jcv)*!#|djz^s;Y;NZQ*xZC&+!1TD}>nFeuV*!oCW8sZ*c zK;t{JLL5-AOBq!V#wUkDBsWflwK2u6?xoY;gd;+Otr*nxreO5{@u2D*>WCQ+E!eoum_Y<$)?O z3P#EO@>!%W~Zx$YA@2CLlkm{pM6s zKp?ZD{{6}EA9`zFFCdJgtH83xD$$(5AM>H|A*LEv{UcJUEtW}MUpT%%S4}D;YO!5U zl{F$5Vzww)5-bbaj8}!~TZ>-2z}CuYY8~*Sem82iTm@EilmMOuUc4{OAy_C2*ZOR} zS7{wwK;mF5M&=S9B7zB4@q9+sl0c@wNSOj*_P7Ai$-IZ>nvp^L;qXjeE{070TY3%D z_&vt%5y}q{KJs;7MNh!M%lk&hKR8sTd7|>^=EzD9TY5bj_~L%kZ1sI=vA+SVHI_%j zm922fX!StR4wv0q>Q|=S+9=~`-jPXW;xpnHu9J84F$9gmyI-!Fyn6b){E&7Bja?G9 zCz?9r%sqGWBD+{9vyXvwj(kXu`-jU7u&m$IAcrn1=f017dIbF!dhZdI{`2Da>~@3; z_%5(#L}94kh|1DK{_o&a_bX3?rfldwWi7vd(vvWX%(dIV?qsyzZNWLqD+p{ ztHyOw?zrX=tzSl16>h+?nB&i69yoM?2!iVeSHIPuAfXK-Gf2vph^3$R*F3?F^4eGu z(s_cK5gBa%ya1E7Jgng$E05mxJCCuPy`9JC8R^CuePmj|V8<-*2*wB>1rDQun`d_Y z?JQ>d=DiQxTM0JGnZ2lH3oRK>a^E5JNTqAm6V{0C*2DvfxDnw?jHc_TwZ;V>D65ifcbh<^&RoMM)oNqukqcLS!frrfBw=Q1!WWNQf^HP~ftmpLQxrXjo}YITgY zR^6SBU>t71yq*a!3G%2l-Bqf*X*J`C4h$_9Dm`6_YGiF(bc@w1suXdIEk~z5Lm-Sh zz%JQAPsbmI&we+!r9K`|y}4sR>xoEahXwoA;BY6q0jxFZ7!mqjV(WM015E)Y=ribj zmFszru#Kn50nqa25-=&cs3|ZbpLGuQQ|6(bG7O?8c(q~`r>rZ>F|i{M`Z^%jnF(%j z+K8Sl=2IAg2709|Z^T>MeQH@Sm1Yn4bw+Q*Dulrv^``QAg&clPd%$0`Q2%qJY?Qy4 zqVU5|!UiIm!uMi7D>~JJQm3iuhN;^Bu7QjdOkrcSUVW(2i7OC8gb@;ck8+i)&LqYE zqP~O^5S<(kpfW~KV-Q>AJ?)Jnc@jPR!{ZUGwfcA;WjfygHeMo8w>3iGh8>8MMq)$< z@r}2;`pW~XKWe3lMXPEyiLp4PokD(e?M@?JR?)nM*fhEUiz9)QC*x^$z;7r5E6Pg# zHIm2_XBlg&O2rZXR^_gODv4J79f?>NG+M+Ro=Yx)f-5%s_%j`tjaPVCtCtJJ#%}_f zlh~Qr_j34n|JOac-cgp+mvEp9pgfjtbB3c-KStnM2lZB)CoJWkysTbu^0FY2{Md5p zc>|Z0`(R>;S8WOvs_=H-NR41{jrR3!eVYT9Lfi z4Jk8Zn3TVmh+KtH4Nv?YaGu8v3f}UcT}arFB2}*E%YFc#C%-FMoesg!q6CXVNtKjMQD!gjcO@rdLzpLv#H1&G z;HmV%`SLc=(Ut7hIKS5jQ{13<+i}OSaO!xDIsiU68DlBdkT~xp@(#)UU&vQ*1LkS= z>Gp4U-Eg!PC6Rw7w%v!JI`vIpb8_@#9^3_o9`EM%K3AHfFgps=4_)qz32UXI7t-X^ z1b|D8eqnm?fEke_&0--U0H^Cj1{dlwOmVM2M(8H~~?M z1Szo)*4545T^#aONvMa_v9`3f3UfO{LadR@)O)~r;Vq_i%GZ(?sUjdk3>Vg(6>jO% z)=%J}L)WbXc+)Q@D@Dj=;R1{gB^jn@ElvBYP|;ZJQU{s>aez5Q=9x5rqAGMRf%EHmQ_XqqW$? zqxLx@<4U{eGsP2DNOj|wB|Oo(0MoOrFC-5tjNSouOiK{ew-9-wW3Wsqr^!cp>jskjeDW?8g(#a~`}5V{#QVexq}?9f zY^sNb)hv{yzxHuoP$7)*kHD3F-dlPnC*z9D*_*hLd(>9e#MHjuHN?QjE4NAPt1lITCL;`2+W? z``~hz;H_TEE}_7Pd;%J`efaja3ViQh6Y(UnUg@&kZosmQ zyJ*62$+7`1=}pnPKQrub&5srx2Stkzut)dB6|r9?%ZO{#s;AS-Fvz5?zJ*6)nKfAj z*l4}e!}PBORmdBWcdGAod4(IR+6ilzx45k1{@ni@O8O}*8@UQB1C1H4=uTpUJBdV8 zZ)>?S6C+T?lSpCe+y1a?RoGgqFo(~e)BPbd60fWedl$y3p97bx@ftjlw~al*ALFmE zZEOeI#k8sCymhWL;xm}Va{*=zLalMCAtvSqET_ecUp;^j1pBJE)R2+@;gJOSh|}F| z7?~jb3qq7+j)#SQq#Wz4RXyjxz3HxDkDh3Sh>TVs?l3z1DMri{?hhbF_ZE&Yg2pl4 zqMr%|Ja1ztGCac@4pZ@AK;;0yYsoQH8_}E_g|+6WV9@`zLnDk75MiW%R>V>$U>X|f z#5an3sH~x*?lH6<%X;Zs@nyIg-3E3iOQ?DS)GFnyQT~XiQT~XQX^gSc96O+w7{;Q5 zNH;3zy{e-J;MpiE?qN+Rg7RDjW*erO&NV<57lKu7s(wffqC3FuETYn}JB>E`U0R8b zZ?QtS5N1OTT3zi{c4LYK#lIi~D;Huaxa2Kc)?8T$s9CRSVXd)MyXE& zyFh8#a&I<7vveTOnGu(eYLL&Lvoa!x`24O^h05NW!W`b4ZW{%@7)it)Zxt3b(~k6v zjL+ioT708)sT=|E8~{JhUTv{7+LeU8EL~aS!UgU=aGBs3{=mXcSq3IO`2*kaO<<)7 z{-|H!Zt24Y%l^r!J`d>od|OYQv{AB8+M!weJPyAlULtq&C8TVaJN*b;fyz?` z+^sfpc&()>BY3R1p)(r+^8MM_8~OgIaMTqbMr6P!=A!KHCYOqnO|=+?;A#~Q?pV& zLJ@r(luUm4k9Jb!UxkF7a~9z8t1jlGS6!rwx#-P(lverTO{7;BqSk!_SZgXhK^Mxb z`R`7}w?DL?ww^l%Zm{jKh^J(F)S&vr*OOW2&<=ezY8>{g=~KuIyaBAWEEy=|@NwSv z%EN~?9H>uy7uYdX8c>1*i-U^qX88o16%ujO5_M)H0xf4v{9(v73f~ZmU70K|1i82K z`ty<8Df^_moo2eSCLu4Jiwb7PVb|+ntY9byBc%k)#u&=qtTnB1 zRNK?rFo|#pn4a0tq9m+-t01rb50G5A8!+!<%7P^|1~^Z5_{>2Y$McrA7fj+QLmhvw z9*K@R6K?U-1dbM$?i+}b78!*i*({eyJJN#Mr zD%TY_)i4nDgo7ThN^l}6=F zd}T^w3L7cCGTi{1pukFHbE;YN%U}JQSmZBd%~dsq+@VpxsFsCRux0lXcH{Wm-lx3X z*MHD%Ib;i4?kZUIc>zK1(ysX)aBg~wr}DD#RuRZkQ8##w+@L;@b(oj8c#n1i2>=uJ zNSe81+v!F4e3dP7kfNE^dii(>qzxVo@C8W#h(fLq8ItErrh9b2)j|TmIMU!iUg6v@ z-n-0d-U^u=5F3?vGT5BK(3oHrecvtl8wh*&LfqHoMs?yW8Bi93q#3C#jB8uPvhzcr- zH{So}_jZ3rH#EoWK95ZHSKs$l)mv}9cU8;1g$d6Y7b(cNnk=#Ft3wRsK691F3_q@< zhj_EL6nxsScVm*SNIptyjo|~;Z_a9I0B~vR>c5{Fu2`d-(N45wM`TN${hn_a zr1*&BtCGInS<+qKs^~|Oo>Z(cPT%h=MBX)(E$nHK48G6EPq&jxLDbu^H(u(2o)Nn&l z_w$NPyTQ@0>+e>+d*NL2rDu#qa%2AX^}3VR>c+hvjGL-xX`T|-zT?qsgX$ag)}Btb zb<;&*Fi65!oRcz&7gqKCd1}?N{U2%KLMTf998F|TJKx)sDXrQ&RD0E696hht24;`g zFFk!NpJqHLSoelD&)I&bVyvNweJ(?S|E&p0-;f%sJ5f7%pkPVkohoTB*T=gWTG!v6 zt(lfDDUQ0WJdhda1QLTSkow?qw@adPt)t} z?faQ-Q%~Sv+Lfl;=2-mEEr8j6c^|7~_0YEr*VWGU%fMOX^oo}>`aW)dNtvmKPyBl- z=_zroY^!S16H%1CQg6%{J$vOYi1_hZ_d)k-hJ8G}^T~=S1`Hm+z_gCZ?yzl1~`++A$ikU4H2~ul`=S@Sk{KzeQH#$Y?UR2RwxB=;N%53AA zsgaY5TOQ}ywWD>1PLWAw))V$cTVnA?U%Z(t!D*jzzcRtpdfBr`mt8L_;oiJig;XgW zw4JrBH?PxGLC8)yseXYTgL+qaF3~w59axkjC*^Fn=>^MAD@yJhJ+D~XR2$mU)z@V< zoUa(C7@;c0^Mw1TyF*Viy{|mCSDDamIlWPi&pEh_8MZv#78ZEUQt$23rdHRft<@X# zUsZTVdA3VFF}{=t-MDE=Sw;5@o$CUQ>}|N#61u;!G{UM0r}vfT$NsSyeOwn}j=Xwg zP2o-b7PEJi=bC3uXi*Mskq3sFUeFDb$wsic2j!}0kM2CXl&Riiv^u&i^_xFQxwXi@ zWSdSE=y&Vso8xqR*{uFUjcbiTqcImH`{YsAimNfpb&K)?<@s3C#2nPrZN28^>kIE$ z^qs98`{=hBj(zOAFU9X#+4QuZ`K(z?M`w(*m|8ZNnzr;Gf7{RYe?k5Db%*|OWm}Sb zH*KiaWl+^YG-}z7ust!uP2V)w#M&o!KW6x3(`ji_)1o_1j{nKDvheoY?S;Saud!XL zjL62gnLE;{$Cbwa0sQvwFK)RoPfxMA0uPT~UlwCY7%p0*-}Up+lH$aAv0I-**?L3Yev^feqm-(35CQ`gBZ{XTuCUrCa( z`+;<_Zs?BLU>QBFC>IRE@LuHQDRpbR8}#^dz=eYk9!e42H}8cQ*6m~ErDc<#-ju>M z335?<-(r`(`(IKXH-JvKH+yi$Fnd5CK|W{wh? zik9X6@tp;Rj#lCup(C{-XWQ>^ghstj$pWIo1jOl1*i<(cKDbeTAr0QaRu??#=9R875Z3BF01b=T8=MK75t$X1#X!k`)mTJRW8Yv;cBVX8XRo(!vUl3LY<{fJ zm3+gMxvqD$PNm5e?p*FpF-7)7pO6g(#4}@UA)V?gZoJZVqj`zV=1$1@*dp%T>fH5&CWfC^Y>%E8@w2V*wF#~xt9o+C3E zgNqLqKBxElv@&I()1W&RU;p;;Jv~3vgN#opQ_jyYD7;Idtr1RLy1by+&&?e9gJAcs zxlQE<`dalh9lsc>BGDRkCjPwP{+8zz`y|Qtr_51ib?EHRZ`PXTt1Y zJ3%R}OqPc7m;XXDx?a^q;;2$+zx#usHQN0JU@`moCwd)c9Np8_PJ0;oD3pF#D75F2 zt?-olDczSB3eOMx-JFmJb9C_nz3jw$e(2LWmKS&qoMo6g`ZK}|3EJmI`;89eKtzqT zHP`d^hHAAjZ*ppGiJ!XR#G?a-Dw#P7OV>iwdot=WrMk?+N|Dlx?$M(tYlkPtn z#{7hT-cuuD#<|k(q?Fz|^_1b+S?U!OPXw-H{!$ZDUL>r2^WwtW_Oypf z_i~Ms$6yXOtM)`ZZm_{#P;9#OqcO+TemTFecilS%%V!++xRbt3E?9gZ)^Ii0*cTgw zp8B-*(8?6aP}KYC*e~1~eEMSHuAl4T=?luUheC3k-utDJ&3Ur^n=1Z?8-E?O$ccZd z=y_XtHrJWwT;g2n9P>phFOM)@V)~BqZ2m;vW7BSL`!)V)d+H1~H(0O!@7kBOr%Cijdv}bP`n_xY+@q+U8fN;Q^6WBap>w%&#W;OFmAB=hiT?Ol ziq1eKJ*D3}ksGmIE-dkUQxn;5Db8R&WvgGJwko1ct}5^TbM37;ez#eK4ktXF*<1QY z-8q}1C`C@a*JBZtQdMl@uQciVuJYVHHldsI^9oIo{qAY{r=M1oV&}uzYNgS(sIt)h zHqUAO*GuWQ+=KZ@n*P4ucK>@R4_7^WQ4@bYlQ(8IOx}j5;?SLK-KX@=KCjrjZHcyw z>5Hc4g)RxbXu4z?G}%L2mP!e`Bc|xF=r@(8Rys?ZrOqiz5c_=n2N zJEylvs1&*8omJHAe-CK5!qz$0dz-93dF1(t zp>pCSB|QGgK)TvQ>S## zH2Tu38)E77G~LdYCrv-@*4}o@;GBZ#%cd7jUp{@T{te3)NTvz$o{<3}<`+gT(6g1# zDE95nkt6kI@lTXj{hiryyJ4Bqqoh@`-qQb57dEUK9Tp zTngXe-0AGL2mfZ!Qq(mA*-~}qnFd4hWVS9ab~)`)e@rp5j>U@ZZ%~npWo_r#$`9G*ETCq=ft5tkvo4GLW8b{?n|0 z_msKAkB_}`^^O7@lUIUw1TKkm%Wl2AYrg#jWoE;y(b?aBOz$dlJ7x}>CN?I2PWou_ z)T{aXR`+~OOgycy;?NV$u8!!+cehTj7oCEymmf5k$Hh6 zi@r9b4ePh|_mz2v&Gn`$wuuR8|BdmUGWV2yqHBC^YN5vP9(i(h4Bl^)mUS=C-Np$^ zcl`_3-bvFo64jzjJ?Y250UPbT@u;dAX}d-WhNqS3r^XbsLgA;K<0!B%D9=CS6H~qZ zDrsCk4{iZ}JW)Q^x8u@kc__l8UlbW&Mn+l(b9X8Rz3_suUZ7n{l6AKBH(Q=orguAg zoV`wcu>KFcuFN?;qjN^rjG#t(;JQ3%FvHm)b7FLy7Rvsu*MtwQ24)$oD8KFZhT`-_ zSp&TVqYW4E!v*0AJEQdzur=l?Gvh`VeJsLl&2cZ#IK*JsrG~f@=If^0hsw(^n>#F5 zY~8aa;3fo^%&#kR3Z$|5CO|pNQyhCkf*Y`YtKd+qC!Kv>LFWIJNlS%*qh&8DGf#$$ zt~H|EQ>)w_JqY4kLg1E4hmAcWcs*oS&xEC%R{n+p4 z7)Q~>+S8%YZnHbmEgK@A(_3c%vp`J0dTrsWGswvd#wL+cWs!YX z{iDh5_rNpMA5WS?Sk@ngqCJKx!wIWAef6`N~q(?ua@UAj(8Q3E7J!JDt6?zC+x)O#5tHNb?Yac3+XD_%YK_@T}H@oVxfK(_GKk} zhhxHi{hNwdJZ1b~j=%DLQc>EbwNAg{mCyl`9M!SWdCvdg&%iDTZfkd}$E>aOOCvRm z{%~zO>rU|KXP_so5+kHNX+O~qc8ySpdRMU?YhhHONfX~6Pc)RFcNSJqbz-XF`ol2Q zz#?Umzrkrd8y|Ea(Qf9B(DaDYK2LdEe>6$fVxd#Ltl<6YV~256)Vg+77pcvWQg6XV zk>D}A6Fg?4!#l=0<+{T>$=XE;^T!)Up!x9`HdeGp>eizpo=HSl^lvOWDVHU78xBx? zG_eSa{)G#JY;THo()G9YnXJ~yV1aJ76~-^no%jj6Poo%!0b*(7j-8`Sg=;XR$%p_m z<2_~WRp$gfpul(ie1tetapq20XW{_oCc^xvzujsum|jxC`)36gO8$<6FkWoqGnfL) za_3Jn-p^t}ue4?<1{L{%GVhvm!cp8n`*4({@V9~AP@Gr8uZ551h6P>Uc_Upw-^UgD zIQ{W`f5cJ;d@HE;Wzc$oOZ>Vrrz3o9JsDtN%~q7_&Ku5~Uj1RUtZIG6a1mKh8u+U* zzi~gMOpW3xvlq_Jo^opx-*%qj@ZLZ(3XuXeqEM#CJ$=$qI#Vw=>31KJ(((%5(z>6HMC<955TMA$&!7vS?00=(W%+YqmAVnrqFoT9;U@ORd&?D+?X5 zm7Y=*?{>czhSs%zofVh%r127n5gv3<=6O+Z+z*fnZH_MKf%D-AA>${`Dy7q&2*b}b zyr<0Vn$kPL{7Qsd-m5uh2FXwzBPG!0p*a(uq zou@e5t_atJ*!oq)0UK&mju_4M3sbB$Pd}V}hwQ8Ab?$E6zIsD(ZkEPV3jxw7%$TJA z`SMmx6nv`2A`gxg3-7QUw{_aOY~zQ~1AgVk>HT=5E6Syh%#ScLS+tAAxJImwuVA4u zWP8Ytknsu@{9V6UiZ#|Xni`m=Jg#$vaRI!(yWawYtL4pI;S&q>7^R6{sArBh_-fn- ze$pmD7B7=e-aIE8uDgC)dG=~}oU3!j_@-u{1}b6urboF(H#B`o?1|<_0@CY~%(;!S z{l~1i7=6fwu=(h5^=Fjn@va0{qDzlhQfh?$sM(RX6mc_1$%L`%%I|kXEHIYd`bmp_ zI$xnjYZgOc&OYu;awWThsV;xhGT`FJ>>1#)1WVR6kD=oB17%*x}QGc1}z6nY!BfIzF7@nz(1^w_V4zQH{(QkXIRlrHw{lzz-=+fWM6UH-Nw1 zQbg5L!Rk8nK94`vul3nVsmI)~iBhPdUCz;ZbWAR!iuae;Q_GJ`(`~{pD9^8%JuyRT zM1pLgYqa>cIXhxP*Yd~hp`GQ^hV|sCVt%QosH6&3F8s0*KKx{Gy%6D+(lm+~YFmOuEi`EnEU@5bV#Qh?xuZ;a zb(Fsy&aTP3*XvR2-@+=%bd4_Q##o_umAMaRO*9JPuL<5(o^^-khNsO;pV=vmRj(7U zGc`G;`DcN0?DJ!N`TtP5W{hYA*pbFs{*VEw{h+q%Kd&d^jkL%RLt0pINE+Y3(t8f9 zQ}tWbZaMCBks&wnq@rL0E_RK!0^?Q}j3Hga6_#era@|yJDU`u4)+7dHTE+{F>2kLv zR~pvd{nW_gqsM>eC{x>}jj{OeD9=9{vqCPYuAg95HK?8Sq~B6lKi!?HSu=0?GsrdR zNb>@{%@^?wrcLH6ubwv7ncIwm0bTwg-?HVnv z5G1Q&6q!foRcX$ym$7nOxh`+`Xr!k%VNo2&7cY+{_4QHQ6lu{5bfNFJToqD8vQs~j z`nl27r;i_}MR`o!5#5qki~1z@2g4Q zE{rJDYvw0=j4W`6O*+c3+_i02E7vWQ00H`wmF@HO#KZ)T@#(biH#gr?=C+JedSEni z{3bTZfNww~f8H1gj6fAMW;jakAt1u~M(}`eP(4egbv^|J65NiC2#1zavM zMimgoP~@IHwAf?QRwh7Hw@BLy^+@T6=Ht?q9KHFQm_VLw;-};M7ImC_Nl+)?nZQcZ z(M*NUaP;eTotzWfZTLyV&jqt4bb74*es`?LAZ{mcU_#YxfroQO6uJ0ol5q{qpu0R^ z2@lHcry32h@JSqyRYw+JoWG=mFLMQ+G7FxPpV~Ifa7sDQ-wIdQHdCQy>HB5qf~KN^ z#|_^fzoJBEqQJ<->!TlYHPSl3srJ+50d-VDt$*u?ZnQ4Ai8xACieKr0 zF>voxrWd>X=N)jAa6)xz!2)?>$W=qRo!={?L^X|czWC+Bc#q+i&R!mQ!GuxV1XzoH zqz@LEdDeI|KIo_39k0ABxTbO_)|IXj7xYj-&C8@+k4G=zJv@(L({-qHCuI78n+&KG0qa6Vz#ss)8`1J)7DANvF?eMnZDs!!JmAfij zm9EvUHLg-;nRBhH%H?<4#mLiekFj;G5qrp`hnIX2%+S^C(va@7F3^)4XMa~1={Ecb z?Yqj{E92zLgIRV?Wx9IiL9M`!FvV3Z&}HxY2k+*G-nuXt}#FHsUxJOa#|(h9lCL-onBV&%(kGzuO3cyAfmDVIsUT)kwO6g;IZH zp=6Y*l+ z8X&0mnRhH(pp#77lL2nngpy+_c}4_4f*oiJ9*+GCj$=UN^BHCO`hY@$^)IwW%%?wU zEONw24UKFgPinjsJdhLMbQxWv_s6>^#2Abyp5;!`eMC#J8*qZ>{U#LBHLARQKOKte zHuw?~%+v>uHGH7V^WNXm@0eKJ)ox(QdVZ7w%AiFu6RI*%*Cx=<3vw}*|0XiZM$NJhTKzYH@#u3o|2ubrTlSL)<6S2g z%={f!oCLQT`~<#RcE@W=!6)V11+v}<1J=(&oxpuRI*|j83XW0NH@fiq(xgKk%I3Bo zCHD5Z4Zj!SN2istJ5Hn1^vRyA#Fj{nczIoa<5%=FPLPwP7|ha$dO5AM@Y@>OkE7Jq zxPR1|R5(j}d|Z1s_{P?3=wtKTvAW-Hf<_-1tMJ>W#&Z16K%o4~_QvaOmwsi%{K*|R zhF}S}&~wtRVEdanF)K9l)jknI>KyOyI0T7L8)+7p%nYa%^b0xo*I`ZakU7ZP<$kO6 zpyw5Be4k;3%Fn-6wYcYf{p5bQ!aC>p!j^%~A3441mEO7@^7k%Z9h0w>4tK8{pH%p= zp{#;ps4=sMm zRhqc9qS{A+9QSv=v+7UQI>Rq_TDtbYc9T$TM7Me%1M4L=U|B5X~aqYQpRXn(U* zs2^+l6#Y$7bcDu+-pt&v!)&;#V3gzix`nf)eM(x2c17m``>P`d=QBg5`!2xpjZsg1 zOzi$UtU=><8%sPUo#TXBHLkmnd*dZfoMv1MNfSF*IELN*+L&8&N_pf$)9{tT#gZ{1 zSFL4+{EW5m9W*Yh*8+dHgBk}=;Nq{PUMwwm{ z;O6XPLlI8;<5fkbiQ3EKB-V`O)+6oO@x2_?uC0EnJ4cUYSjKbH$3ESr2GzWExg$cI zqdCW8MEC<}-7dRhHCj)#Kcg4i=%J{<;-ztPLViTPKMOUWOwH40v+gg*mV>#dqSD0cgc6yJc*0(=@BC;?<3rCD$6n47%<-#QQE85}( zb!T)Y^k2?#?f?ra^Pe#mt!l1zz%i9c+gj^Czhj5~BfX*HW#@xuJHiy}2JcDL5fNd- zg|=hd-n!9M>$>B#Tb<<|r{U1SE5WXJ0c#aS56Xw~&ZNtRz6{qG;gQ{&kIC0kc2>&9 zV6r!=sVmBWDEDlI)w$FN3ZxavodhOJxb?AQQ zON}Y-xvE;zOP18YXMn!=crq8ofIN-YPRT2BqpRiri%FXAneNW_OxO4KD3_%NPi`2l zMbjTy@@BtQZ8tOl4}Yxk#0lP>V>r@Q%sP?_N4Y&Gpo;f1km`8*<7GAEB>H%5(mOWN z>5kW3LNhkrG2ij}z<^JfVy*Y4=R!Nm<<`h53M5>47&LGyy{cBLtz*qWB>`I)8*4BLHvVxd#7{uslvFfMJNXYm;v z%(U}guiMo9srgVn-xa7_4~v=Q09$q=Wlgw^Ht0T+;lBQWFJ}G)biH4sRlLmo-#Mhnr{6}#~hwf zxc_bL*vreSt?n4D`sY*ceE=R;qiefMwU8+aN`F*g5;Q9w)b;nqlBX-2c(?urFI(MkPKb~OiF7|Ll(ag)E)tDR35(d`1S=)W*n&UmJ zFuG7*#!=aFeY@h0(%6y?Zy$9Xb9K0myE*pI_@@Q`zO8&Pw&Lz`op7CWopPOa zb-Q|8y{#iHFo32}~+pb~P9oJo#olQ$z;le-dm#SV;X5Mq%cRg@D zbUkvpMTCeHQG%XTNL0`tRa!We>Cqxa#ELi(?|mrnx10=5B#i#VJjIbnPb3Mw%Vh7f zAv>hTM5mtX{G2j7MJyDlB2D-YG2J&rdY=BMuRO0jo*@<)KAq|NbU>Z-JjD?`Vi=1@ zAM%rmHOtpwj?|dqq(9ycpR_-x*t12B$Q5~FiC8M~y$`bQ$K*KiPpz+UD$@(ZGEpd& zixs{RRXowHZBh>%Zwh?EUsQ!F)*|26Lr#|n{PQiN882UV-f$L+m7+xbym3c{b0%@O z+buiXg?mEhhP)$%Kh9+C?(S(h@#txXyRaAkoy)?*nF-I`dxZa!9PUE(f6fgR_=e%RpA(9Xiza>Ffc{Pw}TyoT`>IE{5Kf)}g zbIEaJ9U;^CR*QIo&ES&b(xE7;FtZu_MPX)~e<(QPTyivAh_ng|v+-q;n89pZa`YXK z#DjMJCBd1UOOE}i^vGj;CHi{|{UsCfIA4U99_NzdK|H3 zA~>7LCC8!;LT2$5KxT2tvGi&z5C>l?iy6$pCCARrSbQdo&lfXT7-rv2KsbL+%wXYM za^wyr19*~uEI50TOODdEh4{#99w&>LY&Mr18L=Bfgqc0XKN1!;ho@tVPjSi7*s&e> z)BNj#vpHD&+GAKoC(lLyPlG|?hw+Gue@o0@PA)lW_uc{^_;17v=HikgV$)3k&+yj; zXM#(Pb60XK!or^A1=9p)&v40cr!*JHb3ARj;Otp0Iks-k2J$?QVS=;gxa26WloJ1!r@?965{e$UNSUg?N!mj`Yqq}g6+}oc6EgqA-DX_*NkAaLG}>nvi#SJ&<>~k( z$On8ChVudT=zgO4hx|#w*@s+m9FEAtBVXj3Ld6XBMJ_ou73boSkN7KM2KxwUjx@)Bj9WN6~Wop!JQ36g|9>6&0yc)lH=wYqQW=$^I|6ZCYK!hJXrv~ z$(_Q&zU3v?x47iUTtfJ_`LkjM`!-}{554&v{)yo1J6v*9q!Rr;ZRG(9!oW^^63(^yjp9{~Q8tn54<~`BAIj?E74D zTs!TPID0K(Ci?-G9LxI$z*|4yUltbj7Z4CL*k5qTksX_YkG)UA>W2U>Hb4&lke@?; ze+fP~*kBfB_LqEtn8E&vOOBNf2>2`hWx?5BW7UuB16TevPr?rV8*HmaLjHyiVR?Q8 z9Z*2Xk9Y}?AA=kR_5=Ab-vlA}w-9+{M}Yh-KWY*)*aC>oNH>PJfd5Q9!G3~OJ{JMt zC;Ta4W`D zbdP}F@UIHa{)tPDyapQGKk?Neg0tU(TW|PAS78-1*zZ8W2ahbm%zn#15l^sx=8_|) zkLdL~?ht17&-@v|+3&G6N{Op}&)dOg{{pFTG6u-M@I1`xU%BKMI7rC9@)MZXA3%*7 zLjK?ty8p%{NA(Iq{*5PLrvDC!pBsY?|DFFt%wYe)B}YjX0slcW{ZB4AHlz{opZs%T z2Kz54w!}mL|3y6bM=m)wu1~^O|LEnO|K^gTGntV8=4t5Xe;_^+lJL_1@IQzd?0?bm zefrA(@^^%Txnbhm-bO6vmiTkTi}fHVtsl+dOO~MMZ1}I_IJVU%+V081!BPP2j>}U;nH5N>BEs21 zv|PxG#0M5id_Ou&m3W3MZ1}I_C{`_=H1wB7<6FHkL|E~^ucNC=@8Gx2V zmEddX5;lgNEdqA-(Mn($5 zFJWg(sng~hJeDi*gBV*ruvOcAM#NgISOGzc^RQxh5-*X26<>=&Ps`92n?Y?$eN!$Z zPKqHcU*eDO+2z1)9nZj&3nc!Du(1_@vioB})MXNXU)Wd?pqxM#4m$b zt7-DlXVF8s#5ba?LgMQ|tTnW0uJxg<(nr9x!0NB`09!5br5I5aE$M+CbiYR8UlBI8 z4$!&X1g(|$_l2FU2b8p}+AOSiW3^e>SvBp!%6eeyB%W^;HdX^{W9t@Z!SxbDF zf?~;}tCslZg`I7ram2P@95oVL6g#Vx$oeO2gTxas-a26Aiw^_aD8Ws!vwC{B=_s&T ziAQ5+Z31>+gU<-h0<;;>!0tG#Zk@!xAna@lpr*(yuu8qeH(G?9HPDK;F9o(q;w96C zooxj+ZkPD4 zgpKV1)O)K7&<=_JtFW=Xfbv$63A|I{|1BJ>iJ*AWUArXyp|IdvuM0cdhnBQ`Bu)27 ze4|y^*?v+2#j1o|32HTygiStx;p~<8dJORZuv3GyMokj0$5dK~r_K|$PvXmf9i-=W z)4Th9o2HdUr0V-YeC`mS#S1&oZL^Pphk^C&ybSDs#P|G%u(LK`k&P{2z7~m>n}m%W z0e1IDnn_sD9VpWd=)``4TD|;u6wvVl#2JSq{;9CDW3;;U#2JSro*yFYtb;n;L|o7& z@c}06>^QKkO9?w7@g_Xi3GB`w37~d~F9z+pfYs#Cb4Mk<7aVi~*glPez9z!hNm}t8 zEtvi>iT_qu(Q7=&a0*Sii7KTxfLf=4UE7e2^*b(+lh94DW^e>?1q7K>MMA zu+tKEgV`?9rrtzj28k@I2O?AX`~4+N&K&cjST}@ zy@V{OL5crf*w`I1S+Z5*>Ng^s-6gha-HO${Eb%T7|BA#9fr9sFb@x!)ki;K>g7<-K zUq{@KDDkqX!pR;G7i}EPs(aPzD?g;2mFcrZza_%hBU;5p2QZYY691gAf=tN}y>1zw zyG8c)b&3C{u(Jq2DbXa{Z%F(KrW^?@Zi~Uo3ODIJaBEF zZ;nSW$AvV<%4n0Yg77U^lvH2~`v`j|@lME)G+?(o340{*ZNSojt?!Eh=9c+(d@qB> zaGJ0PnU@1w1nlG?!Xjm_D#-+Py_)o9lqWO|GXE*8LO{E_2}+mw4`3A%rye0FL*^d{8!HA>)k@GJnSWKd*h)Ydea-lO zrp(JBQy0s8i6kr_AarU89@({sB=|LnCt+Bnz|QVl4+3S&ybj{44A{C8gyqQm46s$i zrgenn%CK?ltQ^>lHnO(zWPSr!1+bfIYk)10d8A2%u}WYMS5X{dsm$MmT`cngaP?|5 zwGGf(6v%vmu(35nfl}H^%VhqKFcOIYvi4!%g)%H^JFB9_t|4r>47bL`)&V=*&wI@p$5=kWfq_kng2*|wgFJqQeRAH z9!y3;vYXLMsm#+rrdk?&Q33>Mnan*{E;AKL<~b{9QX;U!oDJ+_CS0k4g*m1Iw~FOqlzFK&~`f+n?B zDU-|Vp)A2RUsPZ}M$t$T@PdBT5$(o&nWvz~J-|BpXzQ&8!`s<4yl8=0)@WPS@%Zw9vJk}nR{Lnb1i1H0&R>t(M_Z6Oi0jj(E& zZzU6vCQ+nzOOm)1*rp4OP)0Q}4;MCe2vEYIRLpgQ%)bo_k>tmvG%Vgm8NRBGwb34J zBOkO@=HC={c7!C)1$wMb=9?wqVC}#v8|krnng3Q;z#Xp&J3C4_Jhn~d0~p9DV$D^=6&{&*;55mp4uTrJOx+FW^5I;(xn1Txpi2*R z`hdo}!z%}SiN#V*lCvQ5zY%tJhP2{h@?>_&d^eoCJ`!m$=kVBWh@FRevXVoVCCxwJ3sJmb#5TadjJGyG#3h2U+e%WWF8PJtBRXYRy+l;xTp~ zSpJdQRuO`Ivt}C}d_V(i>jRG+m3i$4!p$^}Qs7yJY^H zu(KFIhm+NqU!x?#SuC))KEh7O{5lAHQs%2E%!szzU0z>B=8^bpJg}xsK4u;SM8<5$wyMd_~w;5|D+fDf8AX^OuE7>;PUwu54Yy)sV)tx|zC*OAqKM&^m&$24F|)5u)ullh;)Zlta=XfvOc`D$$D z3}6ir>Sq2W%to4Rg{m=%!G4**_8s=~FV4yQH2jOjXiMI!`WF#!AG3g!HIct_UZzk) zHlTAm3F?>mU&45#uX+d?kom{L#&Q8g_j(;AnSUL&BcR3QK4*4 zwyGn0SGd@6Ky6v{*cF){#x5O_c?DSQs!TF(1u;N0y>(UQe}EwgXh~lNpleVlHdYL% z@1#1#FAE!63Fv5~FTUkj`tY7GSI9WZS;tD^bc>&H0n%6vCA*FBjp!me3I zd~x1qmqp9M!PXOn;sQccFAF=XrU`GRaMXR7yWtMi06SlzhNx;RBAjiYO|SXGow`44 zV;j*HbCkaRz)P%JV#FK16ksL3S_kNA3eD`H%zH7jdK%ZAm00peGXK2@W19foI(w8< zlZk&F?w5(D!Yta1rlFf8mLpBxc;pshhZ>54zQv;Ma zsU}X!rv(vOc?`)&ojEU2*@=d%QtkzBR+^`F61txymF6^v_sQ+Lmuw^EG zce)5;y}(Z8Zp5$(P5js1q}VP=%wlKIw7Mr!tvs?oD#NYnBOP&?l4>hV{3n95vw%`N zeZ@y#5q5SC$b)=JTNU}}avoTMmfqSe3(op!p;~-$cCN6q0UG9tYD}lt1P{l~E&!`Y zCv2t3Yb;&_c6sA{OsB-eTY+7oaa0plYJyc~V}rnAm)yr@E;I4JgE>l__Ekf+tTOSp zgo9n7{aRK7sNBT=1n#Ga|5${vA+(gYlciW`;!E+wY7?IU+Z0XfB0OkXV}dOe&aMGF z@PHm)YvT8?p{qzD+1Yh606f&T&cq`ixo(i0PhX7J<;RmF8E*=iiqaO7pz6Os!MAP6oH5rjdwkUB; z3k7F3nRo-)qGm2R4(=yxvx!$>?h(MwZt}(NGC|TvK=*I&G#W%(O#Jk8VP{cjE8alL zr@_Q?AvmIeMdnk$b*qUVfO!%_&uyiw{5BK$DY1Ye2FVokV6AN|4$!p*f*MWyBf(ib zpd*#OG9j>b0+7Zc8pd`L@4`$Hf%TUXw!`F2r6mCyC{dGj(HL5?nIfcWyzVAuoC0jm zA`<31O%$A42q^Ya6S;yWz7$=g0^8tL1G*iO5G;+J%L?YrRAZge@yynqYOu*}6aS7k zow&jZgA`3$_WNkv0ja$RP*l7z?h znY{UcCBT|0l7a0v@d#i`ft^}GShI{COeAHp&PbnG5Y zsl}&l3W*)k)nsTnl;v_@Z5?C*9W?PWiE19-5iYg@Ey?GbM= z6_*_`@!tzOTLq})3TfAN6Hlk$Ca}#XsfOUFiPvHAD}ZGm@fBsJVeeG}+S^3w&|@Z8 zy_6P>2VqyEZP6w*67grk#?}BjUgrxalwf{q0hQh-f_0d@-dh#0t5ZLm!N#pDtpPndWf_S*(v4ND0- zY2qbj5ym!>D99tj6WNtz?Rg0M#*`*uJZQ(Fu+b)Si^ z1c`T`DN$89*PwEC(k9tNi*y$Dk`;=+AB(gLO$q&~$jSgYcLS@uOMG|U#LrPY6j7Z9HnkW*~OdVWQ z*YFh9@Bj^Nr*91@U(rIkXEy~`FPV58NPCd>_6EWRO}rnywgMY2ArX6-*w@Yu5eH?d z`Hvc4hk?bVsy@thkgE+?Mh4m6KH^?6dDALKi0)CO2!>32AsMJ7mHO3W!4E|^J4(H8 zRYOs|#!%ETv}GlqGlrse<25_$K-1;4{b;&s;&qtFabQ)Oj}Vuc_&H#mz}DsxcHP9I zfOXNJYsqxJVd5t+vJ)h?mJMRvZ+d0QNn(XdSMk^_ledoQ6tLPez9QhenCEFypBhS5^`$Q{v&0Ig zDad*oMxC9V19UfA^=P(|T?%Z|N!5nW1J)00f063w=EIpC0QNAN4BufB{|6YR#0i5c zv)zR7z6h**lgdTcD2q;m8zu#P#{@Ue&IW0NMw9h;*Tg*_!)5BWk)FFpzO9X20k-eL zR!EHdCQ5P+k^Fp62~K%nq5$|+KqqNhP+LDjRLbkIT z!0OHs<~H+vVBMQ$D&wP&TZEZ^6H?0zgCd;WBBohUOKD9r*xAYkB`;gwrm<@2F%mJu zBmg^&hGMjtZ^q+ysOcan*cda<1{2){Hn^J}k2O;;?H-`&E=r`vnfV?vNr9yes2=}) zii-lNcD(j zZ~QR=*x{-~5F^FRKM`Rp5>WF!+N(ZwHAL#s#iGy@+ZZHEN0nmHc%*unx<3ztMPh*U zwW@_j>6lQgg-(dj$}BW<57;3N*!q1`Xpw5BbY485_Ma}?#d&M^Oo;g*q_2$v9nS% z-QT$e{9I_}7sG^|l>s~VP_5)}qw`h33XXWNHI{oBw48`~fNH{4nE984omBuzTG5U- zi_E+TMo%RXb?7#^{$_q&rt>_%6k%*NT6TIOaz*e^HptjC0% z)e*LuunIHZXcu-?Pb-)}QmfL;d%$v=EL6rq!eF(T(ukYs)e>6hHD(?u3(mF>J=FNj zmxY}*04dK`S)YPtTY+s_rs}+8iwI-eh*>W-84Bih`BbC_P3wAz$a_fkg|S8=@`h7d zK#fRM4Yt@$l)b82H@_1Owu3r!8zbUh;;w?aoSf&+Gi58SQD^Aw~5Ernfb5i$jEyzJS}um zhy>kwGf&3k_tQ>XM^vu%F+(%4?*&!2UZltBh!jUH8_oO!VTGI04tw9l#c8A^U$4 zVeB}dtoBm|_oCkHbr?F)bbg0g$uJDX+(jMLsD+aedf{X^JAt-@G@|Y%GmrBn6x-1A zNeflAs)Fo`!p=_70JWS-nj|KnaSohlW+CR=8A=Aqp(x z%1U6H&AbI&o}r;erBMxnSH<*^;=DxGbc5MjjCdAU?rAzFwAD;D<2gXPs@1qq9emRB zG=Z#u+CDPk`q5Oj-4~=G=XZc4%m&{Xvl5813nX;ab8~r^?L|O0I(>(_UJ;yKB4#}9 zLtMha25D0!&}-Ywbl7UQ#NQR+>@vy522#-;Ge3Zx)oA8T=>H1w&N`oGcorLfh*atN zvj!t?yO~#Dp|6raT0%{2sY6Th9Cf^`hMa21(TQU^h zO<)%i)r>Of&s+5F8NyD=d=>1l+q50e1WT>FE^KTVkCd)e6U6Tc8@mH2dlNys%v5E2 z7f`K-Y^mL5uVr=*SXVOz7WSCEWpVdOXla5x7lQl&u}%YpclVn45s>d8?e#0&pjDHZ z@|2Hg9ipiIf1jDoGPtc^;0J1f9+_Mbz_Ko=#>oy!)B|ff#b6GY zxlOoO0)6wws8xEtz+fUCS$9Y+6#g}gP(U@?eM&b1qAi&|^iZ{G?#jZ&Qh;5*KI(${ zKE4o-tldKV++ybQgpH*FD%k4t{d3`wq|v}H+=rw;Xr>5ItC`;fW2O_;8`a49Q3{g+ zd#D{-c@Bmtpx8b6RMBqcezI#%)jBSmj0=* zvqH37jU=BS-R$-MmJ<;&RNsND>=nRPYqi|uLlyzslNzf#Hdc1h%y+_TFSe3}?^`er z48xUx)>Kfz!6`GZmwd->9jpW`IdN@TM2b$lyaRI;Er+tys9HL@Ed$o%p@NRnW*(z> z3pvM! z7TEcE-)a!0>ZpTz=~%RWGo*x#)dTChoDK_rz|8+B>}(SaEuSp>3ue3;&Nf>q|3TPA zGw;TDE}8iQD*dESBoBwsf$>s64I~(P)Fi@x3me-?Pi@&q{ARup!eIw>u#w#NAv3=wk&=4=%I-wdrah`6 zpqlAj#5Q$mwC5^3hTXujy)nb9W`4r#jUCd_?>%TMJ>c_SUlumDmlitPcyg}^P|FE2 z?5>%;p87suNB1;A`}j^^}MuZv~byOrO1F=C%0jA)>|^^|SvhI6Dkzm3sfc|G@MlQmj+$ zL{jQUh-4nMgJe*>9oVr(N<`m=0#uvA*-I7+@lmR^U zz|4Ic2GjVF8OD^I^`L3X zO0vD&7VinmUi!pgG9)7`RDyd3P<l7&}b za#y_MR>Rc~LB=7{PhDA5<6_~z5LT9A;RoSFUL{7(&BpdyXyG~2V0aRTW)hZa;Stk> zja?@tnzjtvD9u8Ze>VV~TDuHJY63@=uArJ;@RRet~W)T6b?+j(=vOreO zA_3K(@ZIwART0Lbfb8!KNG-dqBAi8|X;(!Z)yi04_mWDiz!+la+x-+`&9U&cWOxEw zqlH*M6m}K|Xw?eJ_vBi~@t0d)kP=7iKuMN4XyTB_8E zZ>Q0WwvlKpu)qq3aeE(YpN^)zg=zqxjEX~nm8hmyDU7>CfDRp^uP&oaYG;|ij%zWR zg%%Oc7Sp)4sY#+UdXmV-ve0&Tt8_?CfdNNil{4a%tEw}LRkjVbM2xB=R zR3cA4`w9#HmG=n2KG>t^OgZRxXt&}aw;GN)VfCX;@*TE zO4m4DR4d85K=l>C$}`kq)SJaDRs?M4MMII3&8yan(NM946y!<^zljg51Xi#?EmF%j z3(iV_T}t&CQ9py-NqlyZYL!YXybcs719qZ{c0;e3cY$K7fb};MR%+n|z8ylvkLARC zNp(I8@yEi(DgfQnjK-e{&MIk)>V+t<8&(5Z=%!UDv+(5<9|d-!RGoGo#Ycf1%}^OI z9Y#wPQL*N<(P&v^;q7>S9d(|qMhhunzn;E(Yl|VdS#I&3d8sBQy`hHs{#7_w4H4Sw zrTX&O|H3Q4A?*tt4;``L30T!@RZu%D8qG^QyA&U2ye0z|uF-5$bgo zzJw0216!(D)7!EB+ew}bk~O{FLX`zO0BzPz*MrG-0&2-nxr(CPyMSHM_QKo3#&!c* zSE3ehe*s1)psJI;K+tOT+Q$H0&!x2X7K=Bj)d6hlBQ>p^ z2<$kpvWKbI@C_E81`2h0-_0QV)xwir5O&r@}G|! zuc#QIpXO1nVv%OyU;`mk$f!oHXRDEGJG+3EEqBNp*kys|62>kPwfFA@w%fv&S=GmF z>=K%a)ElhgCBfMs**Gyik@{;9#xB#$*O4c%$HHfO1DjFFBnkanDqkrD(f&v6m_pWWI>Z zQeZ1LQQBg!g})&#Vym#T`}Evt6WqA%;co$|J2P|qceYVZQ z_mf#lT@|TihXas@3BWe4?l#E7BNlJLMIxF83)M5#7w8@&U`<<8;dhy;MS=A-s3$OP zSi~%r0xYhM0$%MF{zbZ@?~lU97NVg&y#@_OEfiHw1vGqFwHdxHY%C4X1ML{@U&9&& zwC)k*&Z3RHDjct5(! z1Gc2a$KuDx@T8HhQ8|FpDN7k$T;mhmw_w-j18R;PakrF%6`U|_yPQnnkvm)ZfMAhE7D~VaG z7+CgYKf>&%^gSAqdVLw~PHd|ZKx>9Kp_@|{UWxT81(y3ztqh{8VakAA(fYZMepUgy z*zI5N55uUOxI3BH^|Zx%2TcXA%UZIA_G%@ttDC64zuV%iCSMJ#Cs)0ys@N(VYz@&a zgN(BtST|&qy)GPVEn12%Q*5Bu!e8^o7T_FIp(Qz%q9teGkJ{Ken*JHene|yH1GgSf z)lDiwJZtfVLxGhHsD+7(u~Id_7WVsH(4xxU4QR`|(P)&vZ6@BVh%mO1zMn^X`W#FN zD?4xT7OT~wsa)IB17w*JoBn^g{yZ$s?K%_1ABL*hRmC}HR~`EZ=wEWq+3lY@c5<)n zOuk9mai`s#j+2{iKTguV-Lcz#+=<(1cdpavbR0XeBO$~LLd*hzCM1EFXApyg#Eh5` zGf03Cl9kJwi_`@(dyKi~NKN6^()w^xU~GTjWVUA2zBc z$TAX`FTP^IuHG$f@~g$0gyv+}sayOS7kp3d4w#;GOf*B(fAdZL$Y60Bf*Y%g-bImh zxc^Oo~xRmPb61960o2yf%XqA~sNFJK?7Sd$G2Sr{A{75+;mtax4(rvB+hZ zwn^HoyF0*kEwVwV5HZC&=9=MgF%&_}3cwdlIPEMY>WMVFn-3PxEedv1{@u>{Nk8-ASRDINWv z$|dFjMLjjp58%qunyn_`KR5j471A;Rd%SDx%Pq`>N1XWm4}6lr4=wVm$|V*#vxX@- zvXKAiF~9zGPS_U~owfLcx>rl=rN^Oos@Gp*Ubit(m-us6SGnP~@{s9-RmQUJ38I-C zdRSprc@ay&3LT0n;_HWKm=RBu97t9XG3U|DgGeVouU9-WDVsQHXxA?b%MNAYL9rYyvmvS1Mn z$ynwayJPIyV<4$|YT{jtD8~Xj;q}nZ;kV;}O>S2C%#$Q57xH*WDkB)$A?LSCBmi2z zVG6N+3(hQ{ogH$JBx{{av@<$GvvJZ-{&VFLXY5QOG0+DpL?i*)h-98am>>4Np2~}0 zkqptaF0@RsG8CE>K;DuRNLrf>Kf?__^`S@wRyOY&?oPGJdW#Nw7hvL~;gaWPJ@p@F zF>TWU^^6VrY#_sVg60+(RD|=U)Nu>gS$=|>Pl2Ub;la5?Ca}$yB(6`l@^m{3P;a8C zh-4Z2IsR)6W9V^#@;h*C0mYp!@Y)QE$gpxFkORp`DlAQ+}`e@ZE{|25cpp!+0-!u~cnhWe<3L_d>Ryp-anOUk< zx%XK(upH-o-uI$aW&kSyR(_ZFxMY>r--ZJVY`crtWvhJo793b$xz8DK$+gN@n44n0 zM++VOD^|JeuOlkS{^|o!0$FOLS;~I>XZdTwT+e=IuY~@55W9+iT%KG6p+J;nK zRchtVrOYb-g>s8_Na_X+r_m-}se{Tl_!?z`rucA)PAc#yvl+^*@~`2<(ik)~APyMULeZaa$ z%*|l}`T^y~nvmAl;lomcCurrWt@P>!IdVl*!y2pX@KaI@*_o*4ArAi5rnSjOtU3+P= z$&Lr8U@-x#bCmId8&>%*c_KkNmr1H~vmqq;H_9cZc<~j_G1_XNa*1g`Mdob05)1nb zpz4e#KC5lLRp!7+z7Nqtu89{gG3o&|Bku+majOn^&hp_qP0drAMTLkt4tKb3Qgy7q ztiYl=&tacudUm5$yCy5iP#!|EK1In*D>qmRym*k%EvsgZ0#wmUsM*S`=OUm+Zy{_8 z^z|{Yy~!fa&SMyVZ(B8U-V;7ugUN6l!nb_NFF(w!`5h}`QA@ncyIcPWyjD9qL^NSS zw^(I7tM!2$K4!5~tCcBEtAN^~`z5HcZ z<^Qc*;u&qIHz)7+;Is0LYsmZ9Zk65ep0{XLM>v-{tlAQ6bAIKllf_Ce!#jY|*HbZ( zyQubVvC9XUBGzry95;K^jQhuN*Sl73Iriz6tP|?7@;v>3OY3?VpkAx|ZEPEPZ?||3 zQRFtMocgWUdAP+PjoL1;0jnlNJ))`DHzk(~%mV2&EHp5?iQiK63%c=u5WW~n#qYjeY;v}xSxWHv{*edTa0PElhGM(9b zRym6KcFI8wGD(}iqTC_^P*NW!?1)v4S(Ox#z@D5VBkB+3->Fa$1*Bn^(cDq1-V}~m z^(qhz+0iCV?6_5C3Am~bGGIK1s!mvSiBT-DjD|E85?e`47RUFyVYrDe+LT+w11q>i z*6Jy%T!(9r0Bogi2k$a%l_{`!r+MA-u5Ly1Wpzv>0*PCGEqq6YtuqdG8z7zMwoYG7 z0yI9$g=Gd-T8d;2j>*!)#xw=Uvud5hXqER!E6TCCU|8*ENGr;ro;SnakBONM?1l*$ zK7giY07~v4cgq8-E@3$ftni4Tk6Ei`Db3`wHSz3W&f}P70n5oF@zuPQ^mOL{t~a zrRznQ<;zwY@fEA~U#~$_d5x1{m6%K9@kiG@otRQ_?|eXO&*&kqS!Ft2_&P^#nb^8j z<^U^jaEESE2OnX>R>*e=o56E$SatT{Gq@NoQ3O$Ap(n(X{El*qV*aY{jCF|;VD0J6v&{Ya(bjoDv{h108Wt?Cy@B1uOTUK3QUJlVjmO1QaJfec%t7Hk-wpE9FD}gOP zAhu)Gd2>}X7i)&{3tnF}Mkj?dE&nm|lG^LIn zau2m{W&PLtbC#^s^YzmpuOb$CAa5oh!!5uA$?)hQRzEW z_S?iFJw#|Ne9Kg05PqoKqLmiQ8^le=^SAMV42AnD{A&87$7om3tmM%v)6y2J!c(`lYJ zM%m=wDz}&f6tirO>tg~`h?t^=r27(a-u0K`m_E}qyNTQ`M%!evpWZeyelSDbeB8{w zD>i*Ze4iuGYYH`-@ICn&UcxjocFj`ROw*Sz4C|bO{KY=!fEE#JbGb@Hx-KN`o9B_$xV2E`5$>71}5DpwMN;foeh)V}}}yFZ41J8RQfnlCw*5{PBmJ90ft6)?noE68 zxkVI^eBV4BmxyMY$UH_#&f8ec5CiD;*+C5X1sfB+VgaoWFuroprfaw3fZg+Ez5XhE zR6sFBe2hyrd7Rw$!1gbiaz-ZZp9XgKsfRC`{hks5t+o@oY$Lbv89-f|gmP^>B}xL+ zTgi~|)j*xNo6I+Oeu6dBHpYrm09D?jzjxIpf0tpWEYg=k(vs1}G8CJ91hq@!QxwvaH6rL@eGzFb+F#nZMd$=ICv_ zd@is&uXFclxUf_!bHKn5?NvZ$Tj;8n*)-SMHDEJWh|OE|nROnpF)#1tEEef}V2jB^ zKGrtL#thW!kPIHD6IO0xl)Ql7)7u&76d9watojVzEKUm+{SZZvWnAZns<3HNykh=_ zH%8&IX5tEB+BIO%wGb z)lfZ0GV9^3m_W>&`JpuC9RETKhLk+&+k1L0AZrQle9pXQAnIey0S8g#3Xm6w(!_;h(Ep%If9o;fd zKNkqMmH#?!Sh3(?kJ4<_P=Ou==oUa@z&eVJqe1H1an3SR08Co)30@Y20;xWHv0`G-QvKP0OZu?4oA$Or6KaZneqA&nvgM z&uN)U7V|cn&fk5&CHVyVytUid_GA{&&NCAY)lgX?XP7@f8;MzlTi9=(5T0DU0Z~acO@9$ySoid$j5PFN@T*Rbo9hS&0{X%$F%K zIgYbf?ViwBCz20T_E(>Z*>0h>-!+5uKDNhJ*T zROIc_x0$1{8nkK7wLM@Z4@{EGEI!*lu%=1c0g^D$9DGx`#Q_(gVSb_^&u4oM?9~1O zUSrs%16_x_uICZ3do~%574QgHVi}VkMr=}<90<2~K{FM>w8K%GJdYuK$(i8gv-@r3 z632sJevix|4G-=Fpu}0`7me9;T0|JIk~!{P#%(em*hygF-gbe^w+sh1yl84!?qG(W z0`};M$s#(YTp|KcON6K6cr^Ank-S6kYfSPJHn{|aje@N7JY|zM*^2Ru=G|k5e2nOn zjW#j{l4)=Izyn53fgN>LFn!S`|Gf$pae(e^J;%_^*yLB0MclXXbTJ;1r3FeJ*qHB; z04USj748p}Tbu@T;-cY&Vm)OduAd2Xt?U=L4IjGmXO;)|7 zTp}H^jd?@G^Pl0W0vg!&gpR7QnwczT)%sA7k+;-O7e`6(r`cv))C&(3ic zv!BNNI)`hD!o9ibE|Cr7;pPGcc|ivtasZ|KD!ogb=ckW1iFUsNFBMSCHM4^usmeuQ zr|*$H?vafX;1ZxSb;dOzYtZE&Qb?Qd3X9%ziDgoe7Paqlh0m61HoD(YA>t}EcBaUu zg6OeL=agN8Xk~!T-V>Yd^q9xXmpF%>+VmN2J{4hrdnQwqk=qGO9q!RSMa1 zF%@IgCJQVoM3e!$uG{Rc*{DyEema@49HN7J=D78HDnwKODu2nn#JWvZ1ZX4jTPj3U zLNw7kV|&FpR}!c*ovS$Dmuc`fY_h@y-xXNd1!B)^G8b44?M4={O`Civ;kwfQ!G3l(>Os9@0nsH>vKC;926jY)!?)WcdW5hT{= z0`z2_7Vm`6)C1kXYSxK`3E2x3zRU481g)$=>Y-U%Jnu;j-Gj;L1=e`l6YC^je;!HP;0^m+fjW%>+nn?iseM)ji!qwt4W45}3i->>%0N1V zpZF#V0r0B4Px0m9r2PY?`=b1lFoKt^zlXV>OHwB;2eH zkPqQU&OBjplnyx**`=Rbhk3aD?wlh9OVQNdz^E7M&@ zt4<5~O&!PjkIF5c(flsuLJblHHcc+ENhA9(7xv_g;0m|}=vg}LM3T^zb=$zgmWU+_ zxr^DfLrqO$^!^LlY1^eKYUW{eija@-&Gx9g=`^aTLN3D4?gMM^vU=A6JK%&(GKtrP zPB^Ok4rdH$WEqtCFo+GXOQ8ex7TwC}h%V3CO9pm@?7$5EjY&K%p&8j*#`3#nF(J)=+j{$VsOP5i!7?pQ>bfoxm=Nj?bsy#6r|P;F(9w z7_K-#{TUptivm6&hl@qq@o$m&nWq4k1cs35JWR((fb49t`D~TY=hMLMJ~UY`Fja|w zRtx!sE(_h|`3#?8!hFdKfl7%aU@Kuf!Oj)3%D17!L4al)On!?&^?Ll@6#-|#EmDZx zHx1L)1K`pETRNaidsWCk*Dh@vvjiZT^iqn_(@N)WLZMGf&jU+I@X!SnV&`@N(DD$iN0E@ftt_Hg$oEu;xCqHXAzjT9A%9O< zMX8V}7{f~t%^vDtsgRfO+n0IyjOkK}bvhT&?NrmH;oqqcaRt!exUc_OE_9LLRfwv3 z>AzO+!vu?Kz@i(d3zdSQp5uZoM)DvTS@)%a2@|r9WU8DJ-rSg9Rw3d#poGi5R5^)$ z@-eHjAi9=hN>)l(l|?<8G3iLhu{IUG#ZFoLi7LTEs$r`f_E({z7^3VHpF>)!s213j zxJAiZY{X$$*%Ja)h$!XfdeHeQGRYLQ{{_bvqWYL#pN&+NpXQ?~hp5JTgjXn+uDT5OxK_w_luJ}|Tt=?~suQFTsi7hlP{?;vS@DY6j_ij-StSk1l z>bM3fe~jS@x=TiXeo0*?(xgBCp@k!t>n}vKd$lu!1ly*tdl2qvH$g`h7DG zz6Qsa-`jiE(&c;B(hgbVO1g&>`KDckiVmvUC_S!bA@BKXk1N`$LPaNJ;|Z*Jy)9&R zfS3O#RCG~!YG!;BJvHlYsOW}lqT3JMVTh>$)&p$n(D++SitgnI-#0Px z58xF8TI@2JP*IpE{j}D7zNhddfpQ$bFhG&Xqu>Ax0@^t+H)0~%5U{u%Q<6#Q_hGKO z=gs*B!<_ftVumPx?uWQ6M2u+lo%OK`cZ4qB9Oaks76`FB!x$Cgh?lnr`3W6byn(qj z&R0w6f2H(+;`_`KouKXZN!lYkO+rBjU^+Ugo@uLL&DPnBd6hPaKI z0k+|-8Lgm~%$b+KPtzu}=llRzek!qQp>wxpIeSjfb8Z*%Dt>Q{pKI5Q>}h;}c`n8` zxMS(Sbajb`T!U)~bqaRZSl|nK?@;ZhJ`|6DRX+N8ty(^(++vY7$D3OaO*$;vzhoNY zF2QaSPXImJGx?T`Fh2#hUt;=|v|#ox0Xy}W)i2#bepy+>UBLsZWsY9L5tB0nd&sT; z>P;X+c(2fXkyiOUCrG{1CrB%`#`hnkIqny_<;411JPsf>AY?w&dr&ZVv%yc7XJS0l zxcnKgwEb6YudHH7$Relu7h;o6)*w^khV>KQ6ReKbUWi4E2>HHpi7h%8UA%Bq$hVYB zY}224&V_hP@Hk`VEjH{lxgpp|?E)%$YU16roO}E+FV%l9ru#m}IIGzcb({6qN3`cj znMIvCXSg}$Vd0;1tQUPx{B?1!TO4wAyXDgZ*(HwnWG_t4+JA!=4QQ;)bSk@s4cbdC z(oG9KlW|@@p^95pvss5tdDnd?g6WlVAMSjDu^Y-JIU9i1#;C|1Z{ zP%aS(C^F9wpr0YGKKH$-C~u)KT8Ex zQ2&O1_AU6xkip?>-N zT;6_mJItsB)_vck5VMnI0kFqST|Ngw-;m1;_d;^XkWoBmG6JCI(TwBH2g1j4i(-yw zrD?7-gB@iFhsw)`^W3dmq7+zdjfX{I4})9==wbn5oxdV?*^D1h*-7KKeo4JADyY;W z{KqxH!0NizwGC}eD{;ZSO?tu4pvEpy1t`%Qm-!NWT|Va;Ya&{#^4C?EDcn=?7MoWSoJ&hI{)q-4dq6ZVGKh8~8aMzV@$;oYOI+ z?Aj9YL-@6T24V?q3uf)!q$W18E__GmHe$DcUGsJ?`Ok1;0Y%;Bf8M})94v10b7$~5 zcKM%y;&VKry#vX)gBzYra1W!nz=m(H;yyiovh7DIM6?1cyzKJ}7V%kpc9&>_C~Z%7 z_JP|FBH97f%rf@5FXSKUk)!w4!AJ2uk#dVpU^%O1EvvOm7C;RHYHsn6gO;$3 zKcMz{ZtvI{!Xi%CnFl(;|6k?dT9{q_XZWi$mpa3v(I(@pDp-v1vW7RRjek?Q#5g3U z>NyrC?ee?uQ~BO$p32$;JisKNl}Pf4gxlqhltrAfGcRchlGQ3wxkcDzJwv%%VP0}` z7ipJ&q+DX=ZT9LU6lLdExDTlMg@Jxqxx|CF8Spf$%YUU@VwPXK)*O<5LIsOC&d7_@ zt}9l#=I^1v4H5Hv%omyi)h@rK%eSuhsbk_H|EYjZ9UEZi9+NJyz{PCYlcM*4JOfne zDB9B)JA>aN7WrlMp;&~ZET_HDAFvLEVcyZja1 z50yRqHX$1CHf3&KQEsutwP?rG@889fYy(@m!it7OyG}^m0XBKV)MH;@z!zBk1rGQb zyR3u#+v9AAG|j=_!0gko#hEB683hi2B|qWgC)wp!lvO0# z^J|RIsdim090pnYZPPsUdn!bncQ=e48E#QkuHI#!sQkZza1qf-=qQMtq^ zNOq$P=^{Qu#5?3bGNg28p;VE;B6P!+47-4Up;6l`$TotZ7!*pS8<*3{nEH#>O}K<74pD zry*AkMtQtQvYgnRa@mNq|xpxUgi|bw`F|U|Z!L$Az>) zDS$GUcmj3K4$sFeQh`OU5X-jfa9tX(=S@eza_sUUo+KUEwQyqR?Q$4nn!&f4V$%2p zyX^BWTxa^vpHQAT>pc_7Ao1r2XSenjd&9loXKb_Oc zRHiGCO-_3VbH1wF;wq5765|7~hv7AjZM;v+!zJ>7wfXezc8PpI1?irZj9aejfa=ei zyqDW}o&sRY2aMb0+c9yuxL?N?E2P?bt14JdSVaBX@#G!$z;i7I)E{}oh9q{Kp-=*> zB+8}+{Bw0wMlWjy-LxCPu56hyybOy95%s`Mo%8J@7Ue0NX`qVs@PMPzF7M-K z8-XQt605SyX7Xy$vJ4Wdw(AbHH#xs=ngfs;p+dwhTF4?_=F$n9w)D+ZwJ3^e^h@0a z6jR7_tXjM7>~@F4l54UTAH%q|5ZiloqH7U#cAl=Z(z?x=_27nGp0a2i*~LrtLN+mDvH_Tt*T)%BZ@%Ugcm)0W?r-eQzpY$i0FwBAPuJpQ zTs%mn9rca1A6eu#Rj?T1tlFcb9_yAxG}xJSF$~GseAC^Zd=U45HF!%QyD$tR{Qlk@ z>p#G`1ynQb^A4e6j05dW5IOTILBuV_`4eZd*tFa(3t^ZiXt_2_hU|IF-$`1oI9^xp zCkt@h6tLEmaHe+J@MH;&aL^o^bFtO~(r+ zWj=j}jeNN^xvjSZOMp^Z8Pp#0(@w%NpK;q*h)mLE(URmF()@C$(<-p8$j4p}o?Rs| zPu4i`N4P^X9@jds!}F%o0INZ#wKUkBtc^E26B^E{C9ChgAApqf8H+>V0v< z>(2D~WVm5;YGLs&Ab{zSP&q+}8;YeD+$n&~qN*od~;b#@lNMFSd7G!aNr zp~nELJk7oRm<5hqh=}FSdFu&(!&^`IOCpY!mwsfT21|53B@Q2ocwBk!_=AtSa=^}5 zVgfFS&fq1B78!w;JPoWa%QsiALl?Xx@~LKx#bfWMGm^z!YZ$^oyNrONk|db|Pi)98 zr?L7a1Dn1};)`Ls{JshiDSVUFO$~r8mwVV)GB9(} z0gc77%xc8r>SS=#s|~M58?HMGEWulW#AaKWeDH_nWf+vo0=7AO3%B^UataRqIlgn; zI*Zorl2Mc~yNrYt&E^1im_{IJ{^~=K1MF_t1KUSX)eOKp4iV>Z$xf~BnZiqp?pAaG zqWBvoFYqswOI(y}2x}(jr{LQHN@+8F$o`WG7MB5)9MpS{bSz@rE`LF}MJ^=SUQPs> zH|LFf@vk)0WG@D zPjiEg=@cxJMf`>A2vjanz)5tETYzFAKMv0p&`>@PDyQwbUq}(vu!Pu*U5@)HDT;v= zCXid``?`e|-{hqUiT?&Cwd4x~3q zRzee7_md~_t4&lG6K{{VC@F4og5@)l_sjD0@MbxeM!S48b-^yr(VKZOQk|YCrA2#-Fu$cnT#?TD36$G0MY5>+ShjVfx4=w zn?v5t@JY9yOv9Mp<*#4kHub4pCI|4vhw8u_uE30<-@010G`)^gSxDQG3$Qz#je^D;+fL8vhXVKV#{h8(6c0F6>yH*t<<^Y{3 zzvb^kc(7!ZpHwa}&#!fs0oXM=t6Clcdc0)_*1xA*Vu8P@kIhE}!(NX7?dO=_$7hvG zECM=vfyaXDc3Qs2fZFCwInZ@K^}cul>}Y!SXMLwzHu)DSL_Ea>yA3qU8#?eeXVq@# z60Nv5fq^lqWiIklMZOHehR17#W+#@K-r*;Um~;TF$lE#YPt<3`8lb9sOm#Lt@ysr# z1NDK{`)?d*xy3sEVIs+FSf1Ndu-KsC*G0OUcKQ3d;NnN>Q{ov7Z`Mb8DjwPq!??Xk z{kd#*9lNBO<(gDxLV?WT-sbRk()Zr7%YUUp#SUkD;+yR!_c050scr*?bC$iF_JGaY zdWz@Ww#)ye_obKMB<}Mm3SO(sy~s|~A>x3O?BtxyCk5Pdbo-qDmT!)3NilIqC*shs zF0m%?h?e+_X&rW#IlhunNOKy>e)KQt*EAU{64-GUOx`)eeY}o0IRR{^+?V%f$1ZQ; zXTu>{NEu@eqi z9|GSP*j_!I*f59YD9m)RWwc2OI1fe10=A!JR&rMMpW{F*utCg8hwOYyg^FxoS9Q~v zaEIJ*>*g^p-d0NFK=w41M<}NpvJ5jY!Xc02xg#CAzrlH2^TXr` zqb>q#O)|8#aQrR-YP#(?t-Oklc$sRVPy3@BGQwY*?4QCfrpUV|ID;{}Lfx6*jS>Uk z+`7b7>PDC$K=>25#Z>qO@`gY2=L((&C^2RcBNpwz4#Oq#U95~^33H`RJipG5;w}7# zJ5T^9DvQ)+F%J13;12Va#Y}RDb!g7hB4CAu#Nr&Xj#<0FMly)SJ7mAVrb1(pZwX|n z$xM_^aLC_MQj~H&MeS*px_=M9nEH@u7Pr4rE>RAsE6Nam{!bMmD)>bzKZ@k4k{>jU z8?e(3&014M1@Lab?m&^NT|DwMVS^R{#~9f5VFs4wPe6AEnkNTPibo-vfRz+6`jqOBf27>vChzENjLH@dw}^S;ssEtdqM0_c+%#cp$27Xl z%aRx&OmoPa4EX}P(Z}gl6(ILvZ(D#J+@`TgcgXK(%M<~nZiQ&7^RYKIh>5Gr18Re) z;j$?jih;Fi=j^HQjiJ7@QO`OcN%l3w!J-pb)gh-}hJ!2n6`TCO;4njSM!PD;Mb|Ei zTXaJ<;a#x0@uqiyMS8uzOZ1efU4ewS!Ee{T$0% z+=_fduG2#X6dglS^>h1v=UnG>} zAZ6MJr`K)69EjcTD4>)zj=?#H=7b;PY>kiLskuWRu8niHzT{C;jzgDwOz;jlB$zz! zkTs0;@12vcy)UL{;A1~JboLj@Ev9inXNozJ55s4hp;|p4AIt@Z{G{eK`ZMJc z_aO-zA`RF@2U|}(;52LH!n$XNr3@Cce2eU>J`3t42g^CsN{~Qi7HKfomXB@#}Bs8Nj^ekT>4efy{M> z3K1)iH4gJA%2PwqY}4GBtMqZaD@Zca8aOhCrcN-8d<(#?SMewE9P$S$L~H=+tMD!A zy&9;`wVy#$zi(0$MqsfvX+8&geEhe3ho-ODqDGo>&`mtoHobv#Lsm2tpgt5kz)tmM zK?|-sSPN9(kOQ#Mg${Yy4i}hC;$@yY6ggysP;Rluk;x=B;wMLJ$}RS}hHcT?es0qQ zA_u_I6HFQD{{df@&+OZP$R!SeUE5#>nQni!9Xq0do;8{K?C<;n*uuHjvS0O~`Z4j6 zi%|wogo_>W0}^e06MnB7KXhQqRL)`=o^Z1w!<2xNqC5;(cZcb}^vCdb0X2EK*O+P_ z4s7#?u6l{X)4~?m_(?jcr4BjHj9XyKbKHw1`pa*_zXf!o*O>FQ0Ob-<#CrJVWe!Gx zqIrjA!`DQWjsdn;%n4KOU=s61e|ZPb9Sc!mxan#04=Pl|0g6v}t;Da`U$$dD$3vE$ z%bc~g}kpBzb zET85vcZ0PK9WzP?cKQ*qI)`Qy%mDU0!JN~vuIntYg<3iDqsJQIfURZnPd#R34mGL5H;ul*f&Jm9#d$X) z9}FloIwY%+n;epvSf-rsrbC87H7?-VyfVW^`wQw_aghUkY167~;+8{hXZxvl#U(z< z#Vvr%4*4XPjJE(QN)3IPpZwq1-AfD z>b}Xo{0#hFHyPCoT@{HyiujSU4ae~m^L2scdJkwj@i9t(WfpMNYCfr}^PANoB zO3n0UWl0&ZoCZ^yIw@4JC~N6hz5=oYa36CCG3hy0Ru+WuI%L^Fppj!xT;}&&h8?nk1YaWfpfh_+Qr+~EKY_;!XzS!F{(g(N=iqI+AQ=kNE1jprsT){=H-7mNV{#W* zegV^cMjSjTUhbpOqueuf*yRO> za*F|ACB6kwZZXJ@`M|88QCLBTfYm3zR!OJ}&4a}-SFFi5!f-!SZgCHi>&Hzc3M=|Y zXgIP5eZBr<@}7im`o=3qPV3Olx1A!3|&ndka5?vUTs zLG!i%CB+0pkH$RY55EWpnKPBJQ1y=4fQ6xl?== zsE^g=`LhK)A_%j{DBS8HzjnT7|U$@>lPQCsd59W)-=Y%^Y8V174Cv_P1N2ZGX&W+%pbY z#Llb7~0h?K7&iAbV`6nt^>{H2J zzJA`pve5%7=2gZ{9%1ERt}dek&p9%cG@FYKa#tO4x$)9yl5745*l03?XOA6n96tC9 zDsVO{0IWb$zfqnzWQ#Y@`AbB599NWuYrSNa9dJ~8eKlR-}a3m-Ne6$Y#-ScCyP zxK5Atse?_~=dFxaoP;Dj%JfL2WeNut>$~xCi&MaMw|%&)LPP}bbem!4C5No{(|P^> z7XC0qt(66E9hM#XBq|D6+!2d-R~+(@Q-z9XU}tpix>bklbZU~H6Y$q!ARC!9u4fp` zZY*CU|BWLlc#v_B?3bGrzZgRs4=iHUm`OINP5_plcZgwJb7&s_)4-;@J+{tb!X^TX zN_jKkow52e?~-!8k4+>Ux|4qru&gs=^WAVT@iiIH-rb}CWf9LDOhZn2mz)RTEQoW+ zAL-aw8NPWcM3a+jL%8M8CmLz|I-`s_ZaWwWNC#AMje~x`%~OcT;7FbJ-7D@m^kM8- zi2B26B(^+lIy2wpG&bic+f0KX=5-c-tX$$8psHO{E{UZy8&LIK#>I9WTyS&vk0mBl z_9x0E&I3w2V`>BcLWPP8?~*d(&CZ=VtjjU+-umyATU_Gcckx(xkHH3)xXiJ5$YRfZ z2Q&F{0rmE9rO@nbA>s-jarvRoSR6QX%ap4Som=5-dG6UuUjw$*$xYg!gAvj^K%GN| zkMpZ4ROADyE8w6XIrP!Sfkkr+U*`baF$W2Z$rS*5?j7}KFmHvxwl_TuL-3YGK<=g* zZ@d=IQ_MM)$(j1X!Q!A24#tW(O`ykE3T)KdwT2VF4A`|U6Xxi)D3>UwrtX+(%WtSq zQ2}Tx{>@`|7I;^_%iJL5Z@zTM1?KSrt9U&JxSF?`^T`VOK-ADe)NoI8+(}Dl>W80N z8C9y~D7Km4+TXy-1=Q}%yKP}@8o%I_X#@QqluOhDx>s)sz_HbA;Be-1*u%+p9V{Aw z4e!tTxOFF-5Lc;Vc4)k*22MRrmb3ka|yqJ z|GWJz+xm0M66VyW8;O3h4QBEVwKR#&?MbJs^4B>eu)ZyN^7tf|e;`_cEj?xP!EmSi zhO&rLP6oeA{bVr=V;inm_N5OE7400Ml{XvBvwW%pvd7sq-eWBuRAGvDLXz8Ic)$Lc za)~ZL-SdVj{k!mgscc8)tX}ug1-9|h)D4Z}K0UxH2S$8`Kf+0`x)+kv-zF##wg?B`|ZeNKglNq!Zt!c_Vw%oLZDmQxs*IH%5)ndZuO zmsq@0)4b2n)HR*LGbK35@phl{*lU&XVINR2qD&^h75wKcu%scA*6`0&sF(wEC7}b~ zy5COmxaG>azE5Dwo)R zr18KQ;=fkG;u)YwZ@Szsnshn0*o36;oH;tXEL4ctqI=NM>$9$%aq2MqHbfVr85GY5 zz`7PHc7PSl(5p;x%4wTEu1$MOxx_9N=3+TUEg6>DCHCkkEqS=*!|_%2X|4CoK$6k# zfU|zz1YEQ5u+M>Y^_d#8XeiMk6)SF+5raS$1RwDmJTpHR2EXM6u$AY7zQO-}-J0tq zSE35j`>K!aAwDMkg3FS8-JBGsu9P|fQRa^6a*~E$5BnI~Z<-qMpXx-b34awVPC}HU zx$iyO$s`-;V#6WH?erx~n)BD~cuswcZMKc4^HmirA^_FR`?g0f@|S25%GM9OA+2gOeUVEz=|e7)DZJVs>Hv76Z|nIKbT$#X|U0WeC3We6Xw{X^bBNG zbtatF10Obt&s^ivu`E<11B>3Q_FYoboH8pwxkL(NX<=OI)1C5baD(~mPq=-_aLP%z zPHDgvr#w|ze*`a>AF#p@fX3kF8Nf1T&1UEt-CtnOrj2Q7z<12#fO$EE;Y?%!>OOwT zlMu%wopZpNE}EL`Ygpm3fn6VbJ!3N!j#dsNiP0u2{}U=$oCma2L=uFvPFCu7v&*2aADovYz%O=08eo>SF z>d36K>28Eyf4v)_OO#R%CJnpqk5#BBv&IZy>bs-Vk(Tc70SPbOAM%Wf@5{TNfX-usAk~`yB|B{c7O^IjWjkxyzG)w{%ifU zYw@+4AW9v4V`=*mJY(u&9+{rI0%RLZ<}E&b5)X$iJF$ocito$6RBq7>$qV0DyGz{W z(~X;NfDQT`LKTK_gZ&a(fW>t&P*fQ}?uDuV`2?n+6{5R3z`Eow|3E3x259TdN2U_x zI<+-y=S+Ti!edeXP`N}0pxAj&SHj1zf}Na2Zzha=N&Wvt7bH)&8R@y=Wbo^%lgAp@ zobo#=Q2ZjpE`Bc{HJ9BYHr!t>Ry@}A7;NvIg&`$kaz^PIBPU#HwX#;Q6@ zo7U%hq?PZ4Q41CKsERk)z3;kHlA5!?iS2Kw7@=8jyy?qUS;*auON?@K&YL{g&nveW z|~o>U=!V>YAAA&NO9_8%x7G;`gws>Jl4;VUM-N!!Fzq5J&ttM_=o>y&+Xw+FyR3Tu3qtnxsagrS`M z7#VOra>FpkRVmx#%dseV9@z3A;}K;}_PKosXrYUWQtl*a)&h;tbwU+R`j?LYC2yH> zw={-S4fI<+Y@knR&<6}(Gh5{>@z;w?y=^lS ziRmxsZMNyjt+(69#0q4idahh^%5QsSM~GO3U^9%V4wX)39>n_V1Kc%!olJB3lM6>} zokMg)2CFKkW<++%16;lV(Y=#o!>w`ZWBO+_=1*9lSnJgJ7MsAbr}-0gPOj-&9G7yl zdt*0~ZGPDDI$&jfvJ%)1mE6n4)q{cC~3g139ZV>}4Z^jT;Wne60fnDCqpc`OuX-y76ER4MA7>&Mzr3y-WXjg} z;2%R28_y$~W~VOyPvT2unq!<1`p10vD{tEEr}+64$abF$`s^a=0%QSnBNd|If)u>Z zZKwRCvWh!SnGHuc4Wg!Y5;UI`vKd%9u$Gk#+-1j82bw`mm?JIEQ-8+E&hoNc6B^!Q zxER>ODHE7JK^ik)-P+Mlx5#0CufrcA&Ox@*N1vue+vZk}ZO(>hrk$clfB9pjL=F`t zjW)T>sauz{J0(M>9Zq@6rrhE@M|p?BCy^QueHm} z2#{lVi%XEr7Md#D9*YVQm-)L1w>*{m!Pss)Gt^B*X;xW<1y;3-+RQ(q@ui-0w(PU5mYC!?*!fb!Fs8r|Z@RkMVe-uW6+ z{j`NoQ_7F*bwvM-3KnI6x*n0vhy`vI5oM9zP$8lmlG08-)Mx;goMt~wRbBy6^-EJn z@CZI=C4b1f9Q;@Ba^MnG5Z!Gv1cBd#>kR1LBeRX;s!;$h#WO5A^f2yDQ zK-5vAi{{?Uqh#qtJh>vM;JRA>g-x#DA?iO)`Z9L9_^NyeOVa?TY^MyD^*d!7Oj;wb zlryHW$=8)zGy$4gVlMQ66MJBnxXC%b$W`&KU=h$QK#ksd&QGdf(fo1dRIsf0LI4l4 z<^yRaZ}b22UQtC^M7uw^;O_9*m$n&Iagsy3g+n{QFxHS0OS?<7^21m1vSBCLQrh@B zOXLH%=alt!c*?-y?(nh^r#zaxjPFaqL&RbYA^nXt@Xgi$>z$EiZQ3G!G{|J_Tn`2e8-&<)?Mw@l?l zA?(~en$nEdg4CKKK17W0UAzgMKlLVbhKNyq@Z=^R!%}C_&9uiL>d!ab%c3zs#`#N$ zi#|c81%FKdGeJ@EYn#SNV5!%6PQPuFUxPmkXu7)9=Z1+^dHh}F7SsHw^*lGe9w<|Q z&2Y_GFbAV_tL_6UoG9{?ip>VV5$4eEb0O6QkXF$fpt(L~X>xL2%QtBXko`^-D&`1f@@qURDTfHJ(k*B4Z%Sfz5;n$CG-|J0rHg2fuY z)5$l827aVmVjYsY1?uku9n9b0#oeCnQlr?QKLd1Xv)9M=?fS7L$|D9cXrF{+dAkM^YSF35Qo@0Sdp97g#yv$WuGVuMiwz^* zBO&CQL(UCfB%dsd-tuFoY!5J`qizueQSUhW2WMI2DOihWU@v-1SmGz}nfc6EzqxRe z)FGBww&7P^gU1`kC)=PG{KP49Ebx~3R;$Jdig;T|kpQe|JC{xtvdkomVy zow5xsXdDGrTUD*s@bz#!8X|>~1a%(TY>|SWN~tR!aBez%Dv;_2S#!$RuPT?w;wN469I<3E`xsDkCp&ng`^z$z+-&~q z9eUP#fwJFE+eaqJ=Rme{>a`Noi~yKAw>S@3pSQ6abD%Epw+a|uUw6`nxCkhr!oz2~ z27S5&XfMi_`M%g+7yDg?sLQMH?SaZAa;YfAGwiYFWJU88Koc!y<>bF!1(ut*f`5JH z)XZhq-eYE!A=vzia*I4bb>0jnQmN$w>$zb_H-8hJF=w84jNy9~yhpBM4#p;@fm;;v zrYq(ch+NG@eBM%XZ1Z)v#(>rfcx1BWBsqHthwqj-MfnL_;`g`#jsSGtkF88jp`w8=_UO$*uOdvQM*hS(-u$@}%LjAZ?y*qrJzb&9ZG)3%SvUDkd8X|R`7CY$ zJ2htVnEp~J(F`c6n#YD~{&LV?w|FSI7En{kn7bx?R9VuNyWgripa}n`|Ps?`aNlms90F_uMb7^8eIZ@2@DA=z*lDosva= z`Tr`n=mivi)+EBS$g__l9_QJtJjQ0Fp9?}C3y@wobwp?2J+dNklXyEo&I20+);iDF z>Px3?1~LR}dfX6mkg07LSZWI6?2!%~FT4kA>uwqycYk>wk21phMr%@=ATB_ofD+>j z%}GDI4+HD>PLQWi-f>`$%y~2WqD=sbngO}rzy)CDJRYWl~ zGH+pZ%`_)$x;eo-$5dcmmSwuVlce)Lu%p*$ogeVQy?PZ#4;NT_9=~%~kUZ_tt99tr z9N*oi|B6e@^C`nj8SP&ww|GbcT+Go83zQWw;tSLOZ+1k^tLzAuctl%rsUT2UM23~g zbc=vW%jgrI3?jkBW6u0x)4bW+C{rlg* zVSSH{i%;Y0CR=%f4M5etBaXIwZtsos%6a@3iaIL z&<_!7^3$at+uHCy@JS7b4$=%iI}lZR8>nnTd3J#<9&%xf36ft?!D5dS;eb$V5W8T+ z1u<=SpC9FbOJ;l!12qS91D?~hPYB|v<8yi~lRN-E9mJ%HLw=_jLWx1l-aMk6nd3A% z6Qr3}UT~a;d0Df+{5Q%iUIMz@ZU|jtgmQ`F$FLz8G6xCYg+B_Yt&v}7D?oSD4+ECd zWGEGxxPI~&xg<>j7qfB0iCtpZY^lF2q$dij{ecOy+`$)!09LwZPP%sCqeK$(t_ZcS zR)i1{1zF0BDcb%UZ_&0}L?2`R1`p$tg5*o~n>fa*NampzSmY)4n*jDC+1MX;Hi!eZ z{d59vpByAVtz04=P}+zIjs3QAiv&PtE|8UK#!t@U*G>bAxMc>3#|nwSM$*j77vts6 z09&mwtw2ifze&JaE}A~pJorlHz{S&1&b43z&t68Xa82R8#!UT2u0u(Y3al|A7S8cc zb(aYs;pwJEd=K^{9oXEwX`B3g_?o=YvMIXEWUbpV^3eE1wnIcFu>KWpvQvVvg|mp% zAo)-5IU#yDVn*%r$|cSb8Z!_!YT3thIV!19ylk1&f zCB@?v>R391c-L%lAV7tPtH4t0cs`vTBzN`Bn3+u1_*K08STWcj<^db5e$_z2BnP=g zK1Ah>KBS}{it8Mr^B-BDv}j?2i-Kc}Kag-NBZxI*g`A6}+-hD6(C0@*z-q!wvKGl2 zi-8S$&x$f$ofUq+a9dX3n$Z~3>66(ImB0pSOi9IO;J^ZUc8?`7yAB-!t_C()V${16>RrPz zHS0b1rM1V{DcvNRLNn?BO~3BQ+yEBUXiVV)tW5RC=;2v#^9H3LUy9f6I)>o>n$2u@3+Nj-wbH1 zj?v&>*BL{%0WF>8=}vZ#WbMrzV9{CT#D@Wo7GM<}9(K2W#%uu{c{@`7wF(w($H>Z2 z;}Z|sgi8=2+G#($s=n=2^-$3P(Lwr~aRb&wc5-xgO=3wObiIpa)%V0YSabtR@G=ko zqhTI)i@T8I*O-*Add$)uDw9q>%L!s>8MDw_qW2h=e3Lx$JryGQIB40vXVp4n=o0-@ zW^dv1)p_&ihUz+&G)VCOarUon~chxoy6@}zRkDog#862nxR<=1rl$RR5|I)0gb zEpW|lBFFK3knGkeE#Fh2VuZsOZ~E2cyy{oy7Nd}jd%J7Rkpmc5(duhGGq)`=FHnVu zame;kGQIJ9cFt*ast?2jM44SbL;Ju&f_Uh~B<+rugZRfPR7_FJE|RwDLXfZ1Jetm4cMkw?M z+gO=CM-}WdTgZP@ZZXdf*m(|Pa4AR*2B~22ke8+0!3{14v2N;DWhA{|NY1v?_tRZ8 z==3Gy#~*P)o7iL+I8c6Fg@{G!(v@g01pvpsi)C;SUEhDLgT)gnWtTb6PQ%xJ3as*! z={othuewey@#=1FGjoI3xN4b8#)jE>z7Ka8P}b{=M5`RzOH5K)_GegXjl;0p;v4#I z2$1*RRv}`Yt6YQci9vRIK<5;&oOSMZt_R6!y3)Wp z;}~5!@HdJL@XB7?aro=|KQ?-9$jyU|acf)$AdmeTFBYqW?e-6l2$qKFv&gswf| zO7W-gq#^3N%0$zGAbH85+#(LxvNzF`8R7B3j#9YRg!{?wD3?g!!=5(%qQ3@Dn(sMc z7JF7YB?hygtr8y~%O77gRGa};f82C;=zP`P!7Y*?YiVxA_3>62fp<#=c76I87M#K$ z`R}k*57M;4DSX(w&v1EhkPO3+r2>0+i&#mJY{1W^0XyADtTafcGo=H25wrdO5%%U` zRV`Qg@Uw@brEvCH=cK!D!5!T040q>bo^EIAOr3d7cPE{j&eZAT*PYy?(@7_}8M+fu zkUkyUV#LvE>P4=L*Kw zMM7~=0cRy+B~8lA1j$y!SaT$O5p(b=hE807qSfSMB_&R*W~^h)!boJmW51d;+|+mr zr7*U|;_AMbp$E%V&at<#I?YhUMRjO9Z8&p831fx1FcDPw==&^5J@45~K4fnx#s4Io zp1!(2iSou6#1*J7)v;b-hjPZUx_}KL>rY5jNYoQk6>!>^uIoVmwV;1ZER{3;*E4pq z3-eSPOL_c_4VaS-HOJ{pflB;e0(g3RETzYIJpEB8PE_Ib#^|cZUS*b`*oa{tQD(-k zIYF_Bq0w9{@^Kx8DWU1ieQpC(* zNOu&iCk!VjYQVNz9>F>ki`s;>phkwO{`pT%NNktT8c?C5E|wZmzzttG39hM+^=wmh zjJ558Ew3TgEB;;2*v@{>&mxwxP@tHxWh+&|AY4N2WbD!<1it0?JQ=nN6kMwk%_E;d zBRDfPYBTU$zvIPjy!`5!ff8WR0_2lhQ-VV@Z zLt;NxQ*`|)DVi8tnVIxB^QbBQh~q{xxAbSL_=n}lf5zCvpt4}*@#*IPB+8BZrYve^)G2iK9Mh;b}X_O$TzIuDsXNE!!Zk3e;idCjWaoWBK*hqBrK7h9?~k zVe}ChnTohL#?CiKCwJZC-OP4KxSqjcz1^TxJ}PyBBKu+3Z1k+EUEsuCRV&qV0mgDZ zt$gN^csoAI*r7-BqOOg^lY$qAZyPd)3uidebHN6a#w?eUA;iG)+eIH{5 zx1UfZhNpwt`?+mv2GssV8Y_N5F#y@L18%{4V!=&=64|`frBgLR;n3@b-pEyOP@HDy z+=1u=*cpb7?^5?atndsmwEU3re5vQx&O%PyfP4AgSo)+BC(dDyU5sie`(hFP=hZw8 zieYX_+6_mIcY0|+&r3w>bTQ5L$5Jk{-37){4(Atv#j-E4R6Agb!#j&f4~We%lOfwM5f5WSZd=r8bu#YDKp~R_|e>y@rjkKzUj8^Tb`pL_{ik+yR zE?!@$GR*(jc}CpuBo{(D*7)eF)3F>OF~JQNqtnrZ#7)LZBUcu;SeLzpuC96_{M9>I zz9?>kDUQJjq%9VHr|ryzkK>aM#Uf?=9e%OB9WNe^g`4eNhUPy|meWlB{ymViSsgBx z@J!!_F1f(d7zN6eJYZakI~qBF zf5r)lB!(I{C`Y3ou@1}7u1iR@(Glwf-z77)u2ZGoz+z5eEVl^zn2M_Lq7i~3mD`SF z!jAO6RK~BTF_icaF-)DYUQMnCI!hlZ7R4bO>ubv5#*1`rSrjQNzbg8ep22NP_bLA! zY=(;&+qqj|NBGN`c<($87hSP@oX6EMvg&1V+mgkI4LB0(<#ftMPjBIyM`OJ}h9!*M zyaH?&T=nBc4r3W9SQOo{bV~ECAFjZym)lOxYm2&Z&o}5eH!bC+`h+It!Gk_1o}E5Q z5zbtZ$4$w@uXkP2GlP1d^71Pa%LbU*J+afpF&L{&14|r>rL8EA!Pv1Q>e%#%rBESb zCzdD%dpE1P3}q!_8$BNDsXHqeYl`H{#fn(TSYf1^%_3w<15K9iG9MdQRROM5+?3O* z{F9pmtGtYDs{={S8_3#!!JzZH^Nd&n#_U}fmHsad9>T<2XvBGwG^cc6?pDGy|F~z9UhQ?TcWcsgp^goV4b;IuzuQN9bjKEzSk^nR z5EC?5h~}fv=JehiFRJ*B6}zGP^~ZXp)3<=$L9vnBHuNbO{1sM@dG(X3 zxoPu75Oy%uvoY6TDJ_PsP^(ccD#U?GjZc~J;Z(Vuo7&fIU}Bw$^+u0N(s;20jM}l! z7gadkO9&Dab=o@lI@ZjWcN#Z*~og4X;yyDW~`#yAxQ%N_->QO1z8P z_6@>oElo!)*GBYxq2kA%vGN=TN&a+R2&5j_gE?5K(pi7e35vZ8rPfUM6{%+TF*I^S zZNd#K5%*(lm37X1oM_>6Ed2MIAU;Y}&?*%2X2uT8S3zYhd^T&rbZk%_k6rw)2N>H@ z^*U)CTOlqkDz};k-1~!!<>jIew=^uiHio*RcCw&o#}p4M`{kbf@1SS12MRBtyr>E(7m8%ytdoZGzzgpNe9{N4$PV;J`Xbt?yh$8pHS(x;P#ZAD!T z%yJ~!n~8Fgq0V_YiJ$ZCn0q0&iWDz=#gU?qp?hs8jXfRaIvfl4y?(H72kuPI$9jRj z1B_J`fvOi``P`wxVFw{QwoLQXZK%YU&(+Fo`A$}eA>8(Fy1aL~>KQ8b zUSR0(CZ!TVXT6B^c3CNh@EscgvoD4h?Kv+}_7c9mSH&LxhP7g7Cz&|ywg?_ZoET-S zt32uiJnh^YC$4bQ=u(`u?wZuCd1f+R;8(#8SDv^xUDSqw5iR%na9#W-? z-x(=g91>&T!braKJ*>KoGd6V~IzP(|OjH{#iZ6O6kO_wFWS#}!OoxsqYT6NPr#AoCAjP0#}oF9#) zV$C73fU){C)Sdb{?m-gcQPlxE-<4_AH;J+S!^#$h^l1wjYg?hN{A#%)$&6L^!sTYM z7dw)|P}K<~*s%wvGInr&FYJi1Ucf*aV^_8TyB15!xO0o*k)?3DUgCG1kVt1FDN8LQ zB!JBTw)g;l^m;6P+KCg3(YGr|H8>WF?A)GNk3=Rn9XK(aWZ|7oyvSl?c*hfwXIb$-j{{b(s;6=zgci3-0H zC-NAZ+J;#fkL8`)Cw}JD)m(<2jYgRzrO1y*@r$!FA7?3DqT^g#0BSu@&dBvV<%Nu0 zzpVZNd7YOtRv5W@_>2=LRxs4G6@9o7OD#J3fVkF72;GjQTRz7Y8!&_I*u%PUXNz6#Wwpx%MJ2yc zGN7)+Uv+|_ilOXL68|S1_Z>&Hc|7qt$-HKW{F#)YJ0#lqo%J`SgW!Jagv23qX*bt+KTgrhXjvf})GD+I>pT;ZxTUOw~%I zt`)&Z4E8IJiYS{Q9>vlV2iCtjSuR7R@PraZ1r8xkiWK~}4S7>wP$AjKBtuXmIC+6O9bP^^iV z(0<0o8)4U&@2BJZ-T=0d^HK4nK{3eKwkQb^6sH(E8Wm&}5~s1pMe;==5akU1txY9| z{D2b_Lku0to;GJyAQ1#Z1(mR`70;xHjGbews98-}El=4nV|Ujmwg00|NStTrTBY*& zhQZ_lW4Q~Vv8^`MUP(f$Tkj^d5EcB;HR6ct=@njSQ1r2J18a?n<$4Uu8*N-9b4RFD04UR z#U=WYE-}gPIgu~)!&-EU-)JAe8y7u4!rKh3EW@ra&*ynK-@!E9RYi(F;Dp3ohB`}C zq%ksa+>1vVR)ys=b|3rj=o6Qv4=~CqRZ{^uStl9GYlMa8=nOi6EWn^pGlIsG{j7op z#T1mwN8pYWKlJ8D45h7B4D$&mB<4NKYcNLz<35_nN-kr|yH#E9Hq=RG>{vvVX^p5d zL9u|FN~_gP3Y1-jP9(smA=U3)ek3v0b43N2zn^toh7vOoAe82(`*3MstSoZnfJKG8j7& z$*Z5kB4{yVHF-F~6brhbc^9L5m>ikU!V7i#%&0P96Eo;rPEcfVQ${ZA>TLq~tFqC5 zwS3^TxrCvSyr*-cAxI;K+bVCt;Y#%f!0%l(fh$7olp}ZpWBA#p)==@sU}Vo;6Em#Oz;@t`&j$srtxa8 z(g}hopjuCsuG<^L>>_=HDWTWP)NPH?^-CV_9?1yP+?_Bqlvw zf$fh@NUTCX@2T7f>v%U_4aQ7=%F*%LtPSJsGNl;6YF5nH;BGIorJugA0UbQO*sio1+5?vizQ}Km|8t zma1#^K_=ch##%N)tIGC!p(0ECbeSdCdTz^&By~E%y8i|&rE}`aq?x-{$yn1lxWVN3 zadN4`YXfkNsEYLx#cTu{ELR&uA2eRZ4kRnv{fkaeYzA|UD5LM+J8@#mv&dHHIg0yf zT=SYNptQxQq(fExaUc0PoB?l#CC|4 zE&o^4*J|B~6Fc~w$%wV3E6UnZhw188cHYa;cHVk!>pu2qW!A55AV zT5`9B=~dw8GmTFa`sq2OWBR=lCt9!;QlOMC_tU$)#6R%cIRN2$8AksVemcU#Xl1Pa zqB;hD&50KWG4JvnDZQWArW~zAp*;!ut#db)7a)|hiCe7mw_7G!@7cfsN z{V2G5n4ya~D%_h-q#e(q@|@z;8&SO4i4Uf$Te(T5UKbWcr_wpT!MZQxS%vZyfiJ*O z#*$KDc`fooHR;Ccto5RtW_k9<9_$SjO21%T@EAk)DwH3{u*n)OWA}%i)_&K)>x12c93yBnQw#biM=mbR{H(h;nWSW`2-~>fKnCSv^&m1r0 zZQxnN#VTubo$fed5OTlxb<^Z2$IDY-t?kM|iS>xn3@sg1eoQ4SYtCT)Zb2-p^3wv= zRE8Kk6ft?hRq8BbM~mPtI^RpVcnNWg^3V7XKh6IV^ z8s32Pz>HnWP?gMFCth4(Xr(7))#mOA+cDE3o1&*%egTX;!vFbk(i3_B6=Vr%_ypDFRvN+0XQ z+_XL$H`k?pS}=okV#fAO3^CEm{D_i$gw>jg>!5N!-DRTBdk&eMfK~WeUerN)SbB^o3SMc z%2=}gHASAJSi-N=T|k0>As_vx;}hGM0zr|(Ev2o~g?{0qo{JIJtmXOKA&8uKv6P{) z^$VgM;Hv#ziL|`ukbzRU`L6OgbHy^o(w3vhR*jz$wAa!)pIXceyx5ghj5YTRMHE+kXRV!ZWwQ6zC=cj&N zDPw%qqN;bX9>cc)KPHe*8 zPhlU}<){2vUT0on1sjX->a+_*22*DXm^7laj72Nh7|kW+t_d^bHpb3J>Z*NG)m4iV z)p&dH(}&ZS9Vu$Ksiqv)9F2bZj+f>1&rVR(a!aRTs%^|v+p$E~?~Y0fwcAfindCdT zsb#e?_2sdItYfSz3F=YOEc&7o67>vSYJsw~$4}k-S_5PAH^P##*N-c{o#=2ReB}pD zyx0X1a0wdn4b6T3NyaWdgo!F;7ETSl5P{c~IqEacGop{7Z7b9r^5>`TkZsYAm09rm_YxaH$Za{+6n}kExH>qrf6E9A2)6^rJh?)g8X)TK)88)~6X-(+nAY(C>xUK}s!y8T}U0t;LC9#zuBV zmlXEm$>PL$%wvSSZBgXC0JeEAF9KgsNNtA8bY{%)kMItrXQMI;cQWDb zf^d-}-TNOW=?;o}*pdgP75WRjx88pao-U9sUj3#x@c?2t8~aANkBX-EjjuS*h{@;R z8hVxgp`G^)Q@lj)WG$N;2KsI?I~?-Ufmu#SOhFI3s=}n^@dP|#tTzwQDu?}ij+rCo zJ7CA8k-#q;%|a;KQ$DLI!{K~>r6@xk=N~aC7BDuv4AL}x7A-S8 zQ(F@fS8!Wvor?CwX*&tn9VI%M%NH_qYOS&zp5qNSnXzkUu+kD|Aq2G5PhWMUNI}bj z>rpL9vv5a{$}Lw%lu;jk#A%FePh2o95zAP!Sj1R=yYekQh@=&a-5nW)kDTZGmBG-+ z?dYOvi$TOA%69QjPEcerbS#n~{6AQuW~g_sx-_`NdT2Idqh7(rCo45B!FZNN)r*uO zhq0zeLYVC;A&eBc+)_RWD`C(@*R2HOCeO3|`oS+DMr*=_MA~v*@Fq|6ZZy zkSKf}+3=?`XS~d+G>C9b>QJkXV5p-cZNuB}|u6#}>?pQht=L@O43R z9G@5yaOD^mh%R2sKX}lgPSQwfUc%U=K}3#s`4JaV%FvZa_2+b$+3WXT}V7H)777v@jRQsTs zHO-_J);_kNOAl}+yRTF0bhJPY&;3?zyYHo#UN)P4J9mix)M|;~Mjcv!r*f35r_Cr>RW-NRJ;`pSLrVkO(n$6!)=lV#o6+IQZH# zT~O37A+Z-L;f{)XfLUrEW80$drGsKW zsC*VbJ?2NYq$W`7g!;FyI`N_z5+@QO-1;~~I4D}U?MNh{)T>UMIDmgRJKyKk3HhSO zV+#!D8PN(7&07>PZkysSp3RyYZjC)Lf5-iv-Kq`5*r{v*h>2)ttR|u;BU0)R7`PRT zc4h|aF!ACr`VvVpzVvaD@u29yte(YwxXn**a^gfMWOUgApXc%U9c_=MZVP3N!W z`@j|jen*NUV8pd;QJyK^qbel6cpg`ZPncr6Aq`HbG9XK4I8yX5Hn>mavELn5SI3y=llafI>arV-`n@1cG9=>(zh}Chn1O;T zecW_1!UErOf})?H>`Z0h|C;ln7(koo$&=HB+lT|5ih%Tn6()jxT)xitBSLXwd|iZ zIw92O9Hz5MInO>aoH=5cv2_oy@9h-ytm6{{etHMGmxiD}khxIRUO$JRKg?aWa?bpY zBgMt%ksSAIls#bDgK>mgst&4psLOf(yoBl8c_Qkoy97*_J+r+s*q0%}E=P%4TZ}SR zJ+$hzyKeC;>SqSJf{7hcO46^DlH`c17)B)iv*-2VKd(KHf?GKC7tf-e8LXyb@*_*& z?U5xQ#TXXpx~Hree{e!#9K(xP9Ov^Ay}{VQ47C-Zq{jrh8#&E9h;f4ACOBp#blpMk zD)1IVTOup-ZB9_!ejfP&)Oz6Kz#Ry=EZ8OH^42g%+y&`J`(LZ#uJqH}oshT(0h9~9 z_>|vE!gimr_T@@f&PA>ZsAP%ic90J#lXxv6KZ}@v4;i~RruY!Wd8a^&jOlJwYR)s_ z5mcOruBBE>P|SM)UTSx|cNg%SCouHj)}_~S%7dA-N_XPLe14-X_lfXHuXcP$Ea0~0 zyGrmk@z)byKeZQ_>6YQGkAR+E0zsavcfA7vhg12M8o$O9mFvYDbw+ zDU4k|4GDH-CY0Y){B0y7Wh-1o7|T7fm<9iHbc3Nq3{6~A*9)&YA(76|j=Ckx9%uZX zn^6X1!~1{@`Ds5~MP9%)#4^Ut`l*2_aL!NbVx5r4uVq1?U@{kjQ0dph{(Fg|FvQytYm;`&HPm z@psWHIwY38fIRHSxF?rQ%3>ub{{_?<_7d&->2qGb-5)z4QSbt44l6(LHB7cb#`=n| zTrT>NS#UW+^CRU#ZXxpT1vqh`Luw;k$=Ky&mE#)*gCfSNQxsRX@t9XJRvsDin;cuL z23H<>9r@Kai(23e&2LmJ#UB-zw2S|!n6bj%>1!-F^Q~p5r9$Ot_@)ylN*HR7=nG$p z=nL_pl$-KSKA{$PduLFTaa-@AG8pApf|Dp`D1U6)d-JDGyr^KLC=%E?6dl;P?gi*; zH!(jm=y(htj%HBE35xaHbngH>0WSHm)ox(uSmcnlgNav(F&ANxT=vsmWH4r|Bmt}K z1+R?7M$AQ?hz8KcF6@HRh()$iNWOyq&QE zkKUtxBz4?@?=>qjz1IneI)+C2RPRxfrXFlisMghC!-*3OSdx)Eh*SFGL*h>SYopTd z$5AXAYve>y)F^&KY0P7d+>#aLV-Xa)8M+dQ;sDb?EWi4lBgGzuYW6;*X#J5@Wo{Xr zf05~U#qT96*vD8#W&xA?&2*2&#eQ^PCrrQ{Cb_IBV+ym>wd-$KQ)X!0`i#dmNW|De z&u`(j4H4sf7wh>4K!twjELZ*X8K^D@Z)nAgr-Rzp{J0o8$k4VUN&}5`f}#zam7s#t zKIQ~PJ3}dZrq>)&0UiQDJEtL(96tO4>e(Ra>vcae6Lm1Oc?moo#{5WqcFOPNBJRXz zb}PYuPIux&7h?}gm7eg;sm#4K8SJ;fb#opCn!#0M|&bu zc`t-yB#F)-cfOCYEnA~5RHf+0`b)eXrFp>!6(2-5a2+e87-YEZr#9}?80KuT z+G(3$fo3deQrQjuopoAlCmu~aTEvM7{In{%d`#a=FCo)Sj3Gr?OA#S<3z|@eDveRd zeZ7sv9=RX>xVj$>i93)^>p;Leep<_Yy~|kt615Yd5coZar(L+^Tj)pmlKc46{rLFZ z$E&N$a7aAhwtWky>-K!pkzx{j)z}twS@vDJ0%vCsas=~w&rg#)*Hhr6$dP=r&k2f0 zn2I5#w7=+t#JoAU8i2NQ#q^xN5*QoE!<|*)Ojx7lGjy~m%AHz@1&rM}_nQ4tibQVd zxLM5_#C=wPeByz3ag{U&d&+i1TgK35oOrQt4h+5Oeq-BL3N{ z)wzVBqR3hJ0e2_|*gdrrk&Q8zvAdDw*};@r%Gj+InCTA9qJQzsbbRp3W9(vtQ2Tl2 z)-bj{4Y%kI{kT^yo`Yg?>a`twR$0qfeet4bm7u^!89bO0Zo3z)7==VBm}I?L@hF>E z2I@rwcPnp@<&5oE^@QVK_AIL5_bbr$Q1t^+ZyjS>7Ac;-#eG|knYxYlRBo)Jv0f6V z4Opj23W-Gkid(! zma*0nHO*&DCthp^*Caw0d;|w_=!6*i4*W-CoVdZO!`x=$EH=-gA9;zaHbdv-rj%4g zZY;qD#xBi|vO0&vPH7H+GE8H~>LbhdsDww#WIco)}g8GF_u|tt%vYn^&Fg`dR=FkL-UhyJo>R>PCrj3`T<#G+{ zH=U6DTU2NUl7@9Lme#4Hd>xWoFjf~~o-J^1W^D27+XOS+JoIQ*! zx%e6{yO21>EnUfqtCRRZa2(ra>SMEUNSt8k@@_1M`4-I+4vCYDrCyz0Uf*$~=mm09 z86iHej1cjn4~rrb+_=DEbpu?voVl`}+qNB@K9a1Ab>hVU7<%N1RejG-Q!|`6G01Hv zBW`o`GatLno#LjJbLvJak`)m>I&dE*5P;J#ThImFLzv$xE)IBQAgr zk$a8zMD8^%g6%fq0;nL?JH~hyKT?cv+wH`s@+^MN35rYDqYkOdrO!E1T*l&^==Kx>q0<7pPT(7@MnLgUB#ZQ})^%RBn5Gf(?r67`Zap zY=ZwYME97Q8q_@+XRKihieW9ZyetDZAZ)r(F0D|ft9(?OVC>jBRVw>qPQ19uQ1g)| z(k|7`qA^qk!C3A*wecq~wQh4$dL++ekIFL{6nD^<)W;ifP~2rGe>?m;(oMu2-D9XK zx^#F*+{eh$qqP2b@c?^r2CS<})mmgyHXq+6F$w$eejjRXlX!?OmrZAPVC``VGIcjH z<0M-&UpOK0h_R#%N~GWB#WT<5S)Rm}xX>%RpTJOVBhyP?gHys~PRnhqm3ec}fxc+5Z@J}ja=)ojXI4`lhxS-{r!7)^J$+4)CPiQL` zTd_{nE=*&FTM0IZ{M|>KpeSN!^8*FF-wBFU3@xm$V<^{x_v~thI?nB2XsLxf#%mbr zzFo@0&9h+lC}wDB34XYYhl?*GUCvr=y1!NJiRlRX!{ExWFXdZwo_k%&*vY#`5U6GG zAv!3^7)wj+;=e8QK3>kyvdf(eEw^BbtYE0K0niEyTiQCzOx_`eR$AE7?rHE$SdRoP*zzt# z*IFnGv&lweQ5H9qSf19unX&2y^|y&U$Soj72YOU$AzF7UXtqcF6b_%;7`xMl*QUQv z%~0n4Yq+S(IB%|EqM2^`U0=R?F_BYf+I<}Ma{EV&4ol+s1lB6 z@!nhq2^9IeAs$3MW65{n6IWq*1=AbA0LL;JTW5KmnmZZWbqLsc%L^vo#aQEd7*;k| zxR+~Ws3LPQzgB5^cPhIXd$?24DV^!Ghq2318MWiZUWVGeQ6w3(){Gj>K1}rZStiaJ zjT+wTgv5TvmiMDaRTgFNVXBF-?5pr;*=Ttl4t=w|tU=A(c0EPS_)#7~3#3&Qwwz5C ziVz%NXvt34!Z%w;9ns3r#ibCGTP!MKx*fEUhz8hJixPNHZNNrxyxwNP?$gfD8-0}eq7xK@pv^V)Q&tvEVV#YFANpo{ zDTGclR<#HPYIa**gm$q;KXs%y!%Z#Y^$hJ{dEyg$E!<@cff#q|`Fp2)UYY8%5aE?i z%tfi@Rhu~n8b_7}uhLeC&$Ivz%gU)9mGt^d#Mv!}roASyP!AhB~7lEZH z4e2{hoEX8{--+{3gP>vEW7}6*`{btjO3-7!1#kDu42>^P6Y~WpUW`IwC4n4G7M!_JB+2LV;O)a;>2BsY7S>a_jX$D-s7g``AWuKjb&97 z%3vva+ip24=2`V(5H;73v$m#JJ&P;L0qOL4nnGf-Y#fW}5Y*9ULA5)Aq zbfob2J1m;wKYRp2A5eS6THY(>%_U9NZcFC(IxSj@;5)|752<3RZ)Yu(p{DYv;mi>C z1(ldioLInZ>62*dvS_LHcw0@x&kmp$M=WaNUL?(h)(z~aMH7g=V=TW}ZFY#oOlGXH z8qS2>*0j3;V|9Z_`rc#V4kwkN@^MJdV-^xXq%o9x6Hv8X@SC?QG`L3sdDYkcBg3u7oN#)%iX7;mI3If4_tvg9GLl-tI# zlonHpY8&YF5I8GmCUwnpLSh+XX}8txy#!e)7@NvMl*kEdI`uIAZ?g)X9AO=;a4xc* zMU{+=7t29~g--~Bla^PsbOpB^h!zI%VkKkkuc6v{-m{DFUu9~h7Wr5w1&7^K+x|D5 zc(K|e?!4%2KW8RY^6=Jh+dyO}oA@t^8M|~t@x-f6oLI|H?+WZxsvLT?5A0sTP2-!< z)N4_`=51z6k(mNKU2%~+)92}GW$5#5j3;*{HSvlrXYBGal^O{Z0V^0wSyo^;Gep1T z-G{9MOHAcYWA%)!$Ny~sHek^P{_Fa z$0s^9lt$cw1=^}~JFgB0W1T~a!BK|;y*#ZLux6$c6xCSo=hbVUXQdVYcHjwBxfbhS zHBO4dmUpwV9i1Ol;STW9*}>SBh+fPxpbkHcnx&wqXJ~m;{o|l$V5n#dcEa~-h zlT}JXy8s_uytYTp@3t6LSsA<5j{aV-ygUBg5T+e)H@Rqe;SPHkyV{N1!y^`b*9nTf znBL=nE_rJFK1|ynetFrV#r$vk8SB}xpRrMkw!!Yo*qy>r#;#bjl_}E9Sa$}ntJAB# zg|UYZfnBp`5%NcXgGTX<>lRhQz{*&2H>A&)<)!C6h#nsSHg3^X#@ZOWy8_q^iyk3i z1Y`Y?v(P~(srbgEk}PoHI*j4As!gDimtzOSY#tPy35y&jBswvPr_sHe7A-@XH0(q- zi}~;g025DZ0^-b84_4N)L$Fsiv`wr|}r3HnMlt zz2Auw$FahfCHa)!yZZQci#Obj3__todCAf%7-92ucsoTb&sqBjBZt9_6N-O(iT1 z76&|^^ec>&R+eB358$5TDi-ykGKR7O^cE*1u3_T)N)Z(qVAVV*uH!>prTp5G05a8% zfsdBr133YjLh3V!_7adjH{d1UxdEEy!4$VNfb7!~c&$E@MR8uhtFv?yQf@V{WdSb- z*Dbu)fIE@=fOl@bjrGw|!^AHLP%rN&ckqd}GR6u6bnuN%NZe&?qlYaIP!nVK7`wPF z2YF%wbpL}+ytogR9tK%g2B`Og9(kuUM~VmB)|a@9$y*daw85$XJ?{j?Bo<6w9=EIx z&`+J9cnGK&^-tCWaB(xm(2oAYa3c=TVkXBU#x``}wY32ls^>Y}tqwpX0k}{lFx0ac zP-%eXIw3Kiq01HBydKK}Uc|=&#tt;p3RV*Ww2bw%M8+0%OmWvL0`xXM9R+9;(mQa| z*z!kABpUr$OXmE_wrA71iXmMrHmz? zJ<0T{3n1-l9z(YqcH#IRpqE&+Wa!i(*v%UP^pN|T&)EG&h3!IdWX9U+fb9%WJuiep z#xh5M?Fx8G_j1OrT?f_}paxbSSAdd94^W#g;N|k!6Yw0LR)T-Z%DLxz1C+p65qM=U z>Wl6Rc!h&kF}8dsu>Ao~6^al9`JLFdVbvQ-{`YZWYcF0YL3&)6A^w;vQ?U0)t2*-%o^f&Tv>}TBb zT6ngIAt|N(jQi(sjEJQ;8D~G^ZsO1QDPG3g&$@THHx@l7pR=EJ>wV$bB0$f}=j|6< zH#}Pq&5?8L7u=78XNw@&(zfTizYEV6A#$W+&vic@juCI5H_A8KZ*o`g-~JW7S-#nR zlbgyP|7&`Se2e`i_kY4M;&13h`J(OeCt}1i^j7&++jZ0UK~h+TZP#6(hi8j8dYgQk z?c(F{^mh4n+jZX_?2ox`4abN#(Z}V-?N7MLvEdl;X8NT3r2Pr^kUtzF-a?;}pRzyU&KKbr z@gn`b{CoS;?lPV#mp&suV}I6Nz#n-leNKMP{=ECEaEu7k7vvZ0FS-|)K5wHh$uHSo zb_e+3?erD-75l3m6TE}ICckEX-Cf8n@1$?YZ`faV=kcGti~d3WgZ+>0N5iwlyXl|g zKiS_zkKaSzlHanw>3%;PBi>8@EdSYl$!+3azmL8xziq$luH;|8pS~l%W54WvHXI{9 zK>s5D#s020J1^1qG>1m-3hPD{c$_ z)!);9$^Wu{<-Qb-5uc`C%U|37?QUR>{S5s^{>J{TdxA;#S^Ay)o&BnNm4EU%`n~+U z{S)`I;o0Kz^auF|`zP+Z!?VQ~=#TP`_D|fmgk!`P=}+=c_NUzzX1OoX|H%KbKMywh zGX1aoU;Fd!3*i{?75cOMv;7yhPz%S3uhKkC&eQC_xX*{r~u*&$L*Ri#0n_b5ChuOMIKM zG?}H@*_u1ZfA%sh(c}`%&e7a|3H!x&C|8rYn!S|&+Y}8rerH3u?UCP&F zKAuPU`Fm8L$pSpD<>&8Hp(YFQT=!?z6`EY3*(){ohhe|? zS1QtEk!G*b+~>l6@grKT$<>;@Mswd5_KSa`Voern_FB#TeAq93OeLBu!Q*$re(@73 z)nqA;y4DO^;{Q>ZCd=^Qx*1_h{5zFvvK-F^{QMtOp~(u(UZ=SO+|i%XdQGm!^L&2( z8Ew$y20RZjSze(^O;+Ohdttx$IaO)03Xfk4`^7J4qb4`v@x5Wc_)prT$xV2i8@9wR zX|pCbe^ZSnYc#u7 zbK7{nenZc_yg_IZazxWgF*5q!@-lMs{3R~iTXs;&sYW6 zCXexRc5K)Zi!?f}$>W;+s+-1iOxNgyCQtA)ufYtBPHOTbuh`<*VM{F5s8^G{n*AoX zng1?Rqdrac@$+JS&eEt~ll_|gez%(YldaK!CI>WoP;*Z+utcL%nmh%dje#7EPHXZs zfCUWXYIH`EXTS=V7+9*&kS2!!6zbuC$kXVoCeH#m$zNTj(K$_?)9g>X%bDx*H5%6B zFo^uyuq6sKIx#2Ssp zH94-?H#E2YyGm?+o#I~Z&?sM*`MO=ey$pytjS6*Hs51*}^@RhXUZdr@T(0v3ueQPg(V)=^ zU9JFdXGS<6c51Xzmn(Ii-t^v@*{o1Wju`Y`N zoZ)`$(P*tM*8*70z+R0?bXlU?r8?^{me{9JnJ&u!d?{>+{Th|)vRt<-boZBGzi85E zoi5ku_7B|;hb_^p(Ry93*Li>VXxI`h8g0h?C>UBW*(q*1jlt983ZcklBX zhc&9zWv$N4J@Wf-Ky+xdU6^y}5AS(nYa{dxBTVZZ3p zs7048y8YkopThysuh9Wr9?`Fg{aN?xVZRvGs9Tra+}j_91LC|!J-Y1C?PI!I%iX)6(Q#cK*X^(J#9HE_ zMkjQ6f}dCM^N2<#b$L>^zszFMFD_}+tIJ+~{%P1RE^E}M%RYYni(yNQYSgdGetyj3 zf4`#9fG!7g-nhORw!~G926Z{8^Tze7a6nws=#(x`>Go;e?dAEouF)A?o&nIo^E0N= zkS>P+O!D~0H9D)yvpQ>O?+ypV4UNv}@|?%3+~Wz2hIKg%fJNd>jn3=xJOn{2cj}f# z7j$_+w=e4MufvwOt5fL1ba@HSES&CYbXk{|b$e8IcfA<4#669!=<V(QRGc2C#vFc{<(E-eyF>98Ca;(lrE-0#Mj{xw5 zrRX%zkn;>X!QkD&5~(`PH{^T(?`GXWrv-*wVAzR<`%*X{7U`5^$RvXW;u`)zx=srX zxe&mg!U2(?Q?enG4Likf_i-N<>y&E9RKrd)Ji(o*(;`DI0x-mXk)>0*A=3eTFKmfy zoiYrWf#h_K4ZDn|(=W<&DmP@gVgCi3UanJxAu9}fo#CeNuq$+0Z^-oqAGlWY z|E<$$gCRHY|FH;PuT!NVEBSFDH*e6X%8*rt{bl#%uq7&W+Gxm)hP}yfe;T$#l}?)t zxf$H^g>XP@)M<+$w}4q{d4!vE+G@zH03I>0S*LA=+y>wZGzXok4OwmQ?g{;6t4=kB zticU7YM2MxQ;z=>m@PVI(lH|$s3qx^UKbvk6oL-^hg!98RWL7k2o@+elr zCs|$4soRjeOqG)+%En}mrnhL>^JNI$ccbBqSK%u2Mzm_;U40j9M$QxAx|6j8N=Jl zyLB2e=@=AZQHG-AjR!@gv=S^P$yPL~aN+2CWqDrT2{okk5g zYOqeg3fX{8R}6W@u&)~K4gQNkovs=3nqglz+*Afm=`?1@F#riXlG8ek8*<#Rf8u_F zwFaGT81ja}`}~_(YtU)JkP}$f{CrlYn})n;*zb22aF@>Mbjy&p4Ewg>Ch-If>vYGE zcMLm2bHB)ngHCr1c^3>&!dun_o$eX(p1~XQppP{No$ed*zF|Kw+@G=LpwpxwC-KZ( zx}?)XLq5dw`@@#FtkaYsrwsd%;r>1A4m!;< zzvAwU3H!x0of1u%XxcBiE^7`tC7CkGv=^Fg7Y}Jnr({zm16apQKdw`XDN_KjI(b8< zR8ywn`5mk{=#*y4G}B&WqC+=zN;hS?$*SC+S#QuO!;~4Oz1VcW%X))OnWoIdGapIr z=#*v3EYr?5-8j}8bXsD{B__+pM4sk*I^~!$$Fy@z_hYO#=(N<7OHDpEHS?@K&?(Q9 zd8WP06 zL2FF8#u9#cB$#!XO%74pfXdI0eHy5BgLR{Q$cO;`(-8nnZdJ50OIgo2P~P`xSZ z@x^ie;xdC8OxXZnl7W1KcA9dh$$N7r|7C$eyG*&uv>Q$Lxv(V)4ccwW-KM?AW2fZ? z?KS0IJo5@%VbDHP?lXD8on~Emr9t~mx!<&#Ot+nZB7>Su*=*V^rh79K4v19-9Wdnq z0C%jgU#vE$)s(HghF%H##TtVSn)0A&&vkzs_KRYJ+DzHTkNm;42DO{A-Lz{pCR#w0 z7<9;#hfMph=}s_EYEXwMI{?6VP-al4DLYNO%XFIQEAXgQ=T;KUejI9tGdddK2!FY zcE9QJk$j^;1Ew4>?LpJc<2AI&pi`ziW!k4r_a0ORgU*=p4BklLH?|lwWXd7aK5M#< z7}#piIa8i9?O~G@WlL-`=)5VEjX@)(95L-nrhAsPkXnN- zoANS%bG#_F8#HRlQIpS^cw>h_S4?>Yz$yM{ok3SkdDUcHt&D+sgRYtKn#sCa19!Z^ zpzEf*4xpTYod%7Wat!m^%Y)x#(6}kb!6TWx^coGiVagk({abfW4C@UBO_*}Rv~Qa3 zXTz4*W6&*A-ZFV#_(Iq(_8N5Cl((7h?QlTsGw6;f@0h$T9OEzUH|VY@?*d3+@;4cD z&y@E}-uYSIZ#L+@Der^L)0lKE20bw41Jj-~VXHV`&_h!`H0>$V{c$)TS`B(+%13yH z;ozV_^L%oik7ZsVQ>x9N1fNXs+50rN7`6t3=KJJ)pS{56UW2W{phTZc1klePJ#0{t zPbL9~XXU}5g+95^$7jG@{6eQe$v&Cvv!8KGpgS0p;*%*prsQ4z#u0;3eKHlm77AP9 zs6lByndY1Cz`Ne7r&59ezZ*j-KutcvxOMG&PkA)telllzG@yQ$? z6Lx}M=r<_WCv$!FQlA?LTVlYVJfFvC+3IYfH-T=3ZGno4&aP>&Y+b(xe~w; z28InP^2s7l8CM?X4O-=ss{pWEzF^R5pInV!oM0+nG-!=at^t*rU~Vv|*e8p9_F5k- z-Iok1@yQZAvz)$cP^nLrG8HqJzef!!^T{$F?}x911LBH7u!H+>-Jtb8xgLa0WL_9EXoF8~@bR&09s}bBRr+Klx?Rm*ybH^n{+!Jvrx`Lr?C>d3tirIrrq$6a1X>`O5zDy#1+k?+NcY_uSi}8gKew zb->m7j$f&*dD{nT0OTu@=!1;`HwNO+{!{JDr#{#ea8uwn2Wsb^`(R7JEjkZJjE)Sioa}|1D)DV;eXg@VJP}b&;Zh69G>I zzP)#obhI^aGT_O;-~FNr-rm5efTsd~I;eiGc4mA7X9Av4S+c$f4V(>lRxM7$&cp`J z1w0q{uU6Y*p$rY24|raD#3J#GlN-1Y@Iv4(2Gwx6U`hj*0$!3I%n&`8+Q8+2mjiz# zsJ>M@Gp&KE0k6v6r^?@_H*hWBwZLBwlmgFa;6}h3GOOzDXEty%;LX6_3aabhQ#&)O zf!hIZ2fn*mlAYbaoq%@&zo~ci%fcBNxEt_p;O_;^b(pyg+z)tPfjeU1^BQ;%@IfG~ z_arM9tC`)v%WxR&B6vA2Yeg|&zWHA+$?I~Nx&xxT##QZZlF70 zw>s-_-&N;kNdr#uIlK(sqn_tzy>wvEV-`-rBS>3>!fNuid)vFq;oS}iY0pH3k z*JzcYfp-Dl1(KdhPQI>z_W|Fl{3Eq9>l^3^*rW0<2xDmAL%k;T20jP;EMGZT9P;J{`iATq`hH=P#%^h#f5`r!9}rgmN%%qo z149lB{h+YwFF)AUz~GRB75F3J3k?hjIVALRdR1SobJN+t(2zqzQF2YGcQi08KiiGY1hE;KMb%>7~mOF}MD;FpCmG_W+}($F92r8M?(1It1#3xyMOijrSx zV0p;pq5o@fY0g}2U`5Ckq5pO@FsXBMt$~#xSB8F7SWS_Mx!%C)kgIj3`U)ex(ZHIJ zYeK&^Y?fAUHn1+_It6~Ew&qp?>qD*&{WoL!n?i2VH=Hb^a<75SAvcG9OIR(F``>S1Ysjq%{O3A14;t7Oa$6`4Xo!r;!v?m8 z+^)b?0UkBb8M0G>9=XNi26lwp5&E5><|I!V*cEb@%0DEGp@H2YcZWiPbw%`P1A9X5 zQ6tdo?^y$TL+%a5GSA6tK5t-O$bF&TA6E0^87~?*5b}TmzbBNTfrB9rhW=2fh1XXN z91eLn^gmbqUqTrgI1=)Rh%1FrylLQQ$fGh565ro8&=s;v=BZ1b|E_^!A&-Urcv$U} zH+5!*GKcP|WlUsah;7rIfp?JVm za*NLmoDF$azxt+SnRl7;~1L!J-)g;3YJ`Ukie@?z+}Q4JRL8W7-8$V)Q! z!+O`&3=D8NJW#FJN$ht0Aw3eqf(wL1jpQYay@cF#l=o%+LVW zLta<;4}>oSxDoP(%D*doA;8U$H|6qZ8+ z8}hE0*dhtQqXOIuc~9hFwsxj9!2OW-MeZx|=l~BwJ_!B8P>WJy0z3-&NadaK#$y9K z4*59rPeLsOj|CX%^N`O&UseA? zI75IJAz!FDX`q@I;AO~{p>Q14#Yq8Pg?trC`ZPfN#^eC6L%t6En^2cSrUZB!@@?qf zh0Tij)Bx{8zE@zhHZ}z43E887_Fqg7@FC;}4GUV2oDtw-$dBr>wU=sUfKMSmg}$>- z^)G5`W(D{h^0R2@uM2Mo&^Kb=s2M$-nG>L2#D0+<)2P0`c4lsX{t^2}{u|X$nV5M2 z21FbX`M<3WiapH_Ffii4$PbE|Cb%HL;E01G8HZ`|l?ww5i8w^=FiC!}D8SH&LnEy) z%PkfM7#49@0Ykx5MW%yagjJH zArWf=v_))-e0x+am0?^PV0^^!k)IH$LahrhG2+BXH1<^>tRcXph?C?qcZf-C2rxO~ zyOpW>y@`%eL|D9@DSUa;Xz>0_~BEPR!_1(2I`va_uxH9tNdMVdE z5MWirRgwQJsiQLo1FVj?I`V5GW#fketc|!<<$Ll;xy zfGrWXM1DX}ZIHJ(6<}+`t&w;Lts|Zeur1=YNSy4?*Up>?us!1T$ahA%e0?^+j)*%V zNe&(43a~fg z-pKEZbZ{>R*dK9!UD#zY+<1&duWhS0i54SJK+!lK|HuUemAE zB(Xcd^@!JX?E7i4A;67@HzL2hA?BBxX8~?TyctOrtc`2W1Kf&uEAn3w*XPWO0JkIF zj(l0Y5VK>=9#u|<_ew?W`iB}pQqRp_)_;%A=Kba6>yy1A`Cm3Ctn;2rE;Y8ynnd+Bn zXC{T1Y&hBYDMka`E;N3T(KWpVAr>1h zR#_%|VTdJ$ON=m|xk@8KEHzxJfV#%TA(j~~Q(%zNh!D#Smm7(=Z%l2?(hw^QR~Y{{ z+Ag~+#7e`J#($@}A+NAJ#45v8Mp9GB99M)`ZMfP<1pgGn(#wTZlb|d-NIqv36#Ah`ok; zRsQ$2Go2y!8SYbAhG|EL{f7Hh{vYey>u+!vzDuuBJ03o!db95XzofUXiA2yxu- zxbY{9_75BkankUl@uy6+Mt*Q8#A(CR3Vf$_=5UBJhG&dFYxJXzgg9q-&iM09LpvJc zg5d>~CEe``anbOi@%Q@(AI#0M5SI)uDX>m^970?+ysUt(xSa@b#qf#|!f;xq|73`( zhF6W>(x+M>FLf%!HN$IS1M0F)hq!Kd-AFonLgwmBh#Q7Cj70lA5)RIWxM_G(UDAIM z1`*X&P4-iH_$b6o7dqE4nKL|e?ZSQk}<8h;4U9L&y-b8;-H?|F%w{Uc0?IVJXgQvG3V z&436~V@?(Ew`*$#Mwk|JnuxzD^dZ9ZnA2nbxR1C&YX(P{5pzcDzg7KJ?aYt}Gh@z- z{nvy%STi)jteCSz{ttC-hDDeib9U@^ebEu!5@AlvIkE7Uak^X)VQ$R1v7Z-L|Fm{y zM1=V<=j#yuN$t$Y2n%8^Q27spIYd|(b7AZ^HmcvNt!a(0DCVNre@S}~Mn_m2bFu!s zOYC?|ge5VTDDX_Y^w%kn9E{+zfnz*IcSTpJmzu*CZxh0BCLqH zLV+upaEAyhW3G&SOYf%lm=IxA%vG_Z(LLJe5Mg!9)v?fQU0$CQVNJ|6s!ZC@IXS}G zm}}!N_(yK0L|7MdUF_G#&2I9k5jMo!pnxuHOpDMFvqOQO5c&{dW6X`QZ_&Ly&di9g zDdwiwzX`SHb7q9iF*nD4i>Q2VW<}Tvv`QfkC z&diO_8M8C??Y*im)z-|5up{P<*ne3olk+3&jJY%RyJB@{3nJ`}xjXiI;_9fjMn>2h zbFTsqB%UmaurKC51#W5uBEtTd`xRIvX1655ftUw$=+wt8jc_pL!C3sNHa9Pea46=X z*ng*L7aLg~;c(2uv9FpzZ$*S7F^|Ol+trSw&dtgQM`Iq1eOFw))%M5;$6_9f#RY0Z z#_9;iV;)!k@^zsP5l+NB5lhN4TGuTioQ!!g7A7xzWL<<)F;A)d=XGw@M>rkxbSy-E zpms+_I1}@Xs;M~5jtFOCo{jyvxDs->F~a$n=VOWdTE^QH;X=#{Y6st6JF_{$#h4dk z-@muc#Fhw`VqS{<MSRb;2v=iXjU^sU|1Kd95w69&7W?aQRS9{B za3ki8SVDsk{~Zx-#=IH(TXA!(ZfAtsF>mWs9FVEl72!_IJF&mptJ)x@u{*-un0I3d zSnV>Bdm`M6c`x?QdsUl%UI;{l`!Vmw{$;OfzI@ld2oGXDi2cL3`Ul|;5gx^Sq_g>0 z)cin%$1xu(pfu`WgeNhdsEZJ4e<(tC%5H=jL#Pr!k+ZtQ})VB0P)vOfQ%x zZ*?@n^O(iks`*$0NLs`8xK?dsj=eO)|op zm~Uc9K_4oGi10S%TYbuQEkZT#g#8uR zsa1#w0}>8M#Ggq9e>uXygaZ>lD5`!_C`5!o2?r&@LVjB)M1;W!2LRAOFTblS4)t1u zAqj^hLQ2HpUXL&|;m|~kL%Sp+3`;mH@qg1?AoF?Kg*Ur3&Fg@Y)#Q$|O>3$nw zM#341pPA?zy^Am_;jBbx_C7Ja_Yr0%oSldhJ0M=IC&HYBa}wX$tC}R=_#wjFgmV=T za`7?3yoB=-F=wsqeu^+Z;d})iDtCymAmM_c{Hb3^iDla8=@0C(UhM!wl9WT%&-fYm33!gliLjAku*xZm=%l zy2P(fbj#`pgAEBcB>o39LX9-&NZ65x$^B-Xn^6WE6K+g|<-G`MXIc$5CES!q=>FZ> znb8KD6K+oYmP8lD#~5r)xK-t<&dpeZZ3(yO#C#T28E3FP;dTY?i6d?^=uFs|Nak=y z+-SSOj)XgO5UxS^gTc;(I};%klOzaEFxZuFSK@ak)hD^>M1wsE_bBj6w^kVJO}ICa z{I%26&P+DgmvCR=%Vtq^ioyPb`x8m~cghu04Gttckobd1)5T3QIF#^^0$R|TZg4o^ z;lv+Fst$Sc83soa9#vqa=-5nyu7q9cOSDiv%ivhTV~IZs_?AA+ ztY@{s&4f4g7w2UH))?GMcuRrR^44n&ZYR90=Jg{&A`I>%yp#BAebivr8{AEJHxcT- zR><@QgL?_@CH~7=4d^hqpYVR-A0(Blo!MycFyTX$f3eQZCWA)_A0_^A(rmBXZ15!E z69sOHxo$D&PS~CJr-^owZ8dn7@R`a#P&>2D;CaI5iO?EhDccQRBz%$hmx}B*Gwd_?nDCBeoFl3=4Tu*=$o={>iebD zmxVzX^iSD8_3L|S2s~skAmxD64@|Wr9fPUY{dojGgJma;AN?P(>~ zpEDSra(pVPB_Z^@!Gx3(QlX!^a&*C9V#UUvr^7d;I=&Uy20#} zvlUn(x42<2C*>Rk{GWiUVGde_uQEz+idG<*6u}%=ANp z6)9Jw;?2KDh=ak(lq*xeD%J4w*kEc7*pzZp>VL0UX?ba|IpyZm zA877Ne`T;G~XPl5^4fcLtp)J5x!= z{#W4)20K#jNd3EB)ocmRJq9~d?o548FKvbUV6ZFYuGD|*ty`2n8thKFJN0|gCh7fT zus7vi1(fuDHrSVPU+VX#)l+$gzA+A@JdpZ>srIV(i*YFBp;R({c|`vhhf^L-#k_@M z42W?g`4p9*7$aWv)8)Q^g)zo?xV6r(F;SL(YW&3gvNIF|BQ>c83Sz8MnZc*^6c zKar{_4vld#ANi*YsO)l?Y5cG0r77}ru>OZ|mr%SC&P>nX3N{%@CV>lawyRc#!ge0@@xvJ;uY74^wgVS}2(j z<59{-3ak~6Ff+#El#f$CuOal%nprWPq^-DrF@n8*Qv&1#8%aRy9DSJ}?uy^&W7OD{AL&^`S|Cm+_ z+25jgm3?W$c&9C3;rJ${77K_E$ie_No{IG7iZ6 zz)bzv>KKDE4$A!Ctol>o3NeOc9FqB=S+l)=ZH!?Vhh;)}P6)6rMoY#P1#}T+eT?B5 zhbz#nC5IRzGLFb3GyI)8HytrXW*n*Vc+s+rF-Bz^mHB>ss=pJy5Ti9?tA58oQM1i4 zMrRzINvxK%YD4CO)$mY^Ss7<#{%&vW|2!OHcE;J6pOY!OJQ8DW#<`iFmkH;|&CwY1 zGtSTaf~=Xwcg0wkaiIdL5XWLH%D6~}UONDf$5@vBbm?HRXcen9X=t7MGMjGdVzY=174A;yl3J5;NOimSUFV`s*lnV;~* zR+&37c4gd^N$#Zi*WDPqGwxPki0Jvf7<)4A$^71|xxpgFzKr`ae<#%B`$3HT8Tacg z$7<6lm{--sDb;mfG z@o45dn>|xcV{~Qg%KV$K`bV`h&te?Qcq|jvBJTfrjN=)PXZ}moCGB>IaU$c1OlY~- z$jcZfGoDl%`Ks`S7^gCx$|L}ZbAKJb(fxoJqc^~6K#tWIhm{tE>otvH*moi?;{P{-pP+a|o7?(3%R$#jf^2Zog zGG592)vVdu@+ro(jMp-MJ*%c`vqOv<8E<61C9a-ovqOTL8EzCkG###-0nYh6T z9%OtVvYu`;B*DXs4@K6};m`z+GCs;A*z3aeumq1YKF)l5)GXw-BzThXN#?to8aq6} z(~M6u|JF2v(uf4lGCos4^V^XLo@acnK$k?pQ3+mTe3AKI(B7oh1TQnb6g!=!y$%Up zWqg(S*I9K&((o|}-ei2EfG#VIP4G73+swbqs)cgNxCHMrzE?oE^S34F$=H+m{b99H zw^=0kknuz24~0qv$0zuh@nh!q#MOEknF$F#W&EU0O>}Nzg3lR0XTGo1lw(qYewO{L zU)tOlJUKys%l=jzmhiYK2?kgWuwvQ5X{II^XgSbIlJ->jLxMq;gRGz35GBmb^aO(~ z2U~x)S2b2tW=4V`mO~U+CwyRLf}xf}tsiE!^f)U)i)D-T?Lqbb6aJ83xaDx`M_4U& z&Pg!Ra-_=tq|VLU1fwiRS;+>p;c8xjR?Aiew83|Ng3*?vt6D=oNKd4vr{lXp+OtPG0C0A1~x-!9J%gG9;7hRQLisck5{^8Gs zJtUZFIn|1P`g>sy38q<2voe>OyRA(y-Ez7Wf+?hYU4j{wGpsl~aX9M}%(R?o#o-AZ z*^pqCw3wC4-w1%(a|r{ryHYOSf4hm}fap0oCFy z3FceQx03Yg0V7)zEU;W)#bsze_O=8IEf-pUxl#3(vD=w%<-t?|vIuk6mTx|XG zUe(>Hy=s^p36@wcQKV(Ao&%L&spV2DLCn>;*_B|K`sjMng#>#n_gKHzRyVcXA;CV&eb(={s_2&z9I!lK{cF>7QkN4Pv^=Q5|1I<( z!6C~-)^F+4JecWfg2R@Ft?%rk?Wxxi9I-s2GHYk9Cpc<()cP)~i()qt9J4&8hM*fv zZzecydEEM!4PErVmEeTs36&*K-%fDS@}%`&ul}KS=1ziBmZ#LvL>2BPIBj{_O0xRH zb#Cq@IAeLnO1v3q>fGEHuUWq(RzLYX!F9{)*59yt#NLYpH!W{kf6HnI&dUV1EpJB|2Nu@|31MZ%SRdv zbSq^~g2$GR6;R#ykl=~s6CLTV3w=n?ZP{)8;6`(&*rxlhe!OoBArCa94q!{8j#7RP>8SdB=Lmh`YnXF3^fX1a5<~Ypx zwmw=tXiL%J*y3cmwb7+L#c;>r&Ug2zw(C()DMmPsaQHha;n?5C#M+gINC{UXibGYq!{Bk#`&>M*Bqy&80R=nWg#Qe zQnWd?Ip6NI?P+?7@s8u2pWrm{n2}Q?l@gONjJyOOEJT7hVvsLjS}-y%ygXT z{47`fU~SEU6tf*?%aDlkTbN>w;~eMD^=SsTMJeVw&UF(0#tGY5oMN8iJSX8#R}hw@ znD02>`5TSq_Kl?}7C0_YKoxXZiiM5~ogehYUW*iq92Yq;8gbDpQY>~{>_p|qD04`$ z#Bqu9gTiJ%@v0O{9hW+>JIUZyr=%!wiYS?$c46w4i#JE4hUL~B#5a9rWUM&ya> zQmk}b>HI3EX1YGbYRA>iuW`*YM>nKc>$p~dk@Ccj6zd$l67i)5}HN|Gf%_@t2Z%eVoaf=fVyH_sQ zo?@%xR_DLFIww=znPQvcHYY6df2nh`BgJ;d?M?#O2xSZ@IvqQm6c0BeLlrS$BWKiautL$q`2&O+4(E3IxjBtVv4Jd zSDnA+l$KmdaozE{%0FM{=5mS~jyIfe-^X&tD=BU|-gJI!s?CH~Q`~aArTX%l!WmNB zcD(KUVWS(8uBW);ct>AJckkXvao6##6V~}b&w)yD&+(q~_gyo--b(Sn@qzOXol>dW zDIPgKa{o2ozLVmy<6|eJbo) zX76gEOm}yRH;!-gy@vfx?ab2@Zynz{p}xN$#398y$9GP0beWCkDc(E2cYb5oTorne zqQ|jEM|-!P>X71t;|C`Rk)&L&Qhap$sL#Bn8!b|Na{Q#g4Bcpv;H23=T&oDgaaDBy5qGSUyjL116_xpQmXWhUIBXf?-{lTWC4azVo=crr~EFVxnI%rHLZ_}qU>UHzyG6LL<-{qTk+jjb6b=A4-O zgUvRa(HSP?oTT!)cGpFf-@OTy*KB@Q{faX62lf``Nip!=wyz za?Z(x1M10*lQYcCIXCy+y{aWLMpH7(%Q-LiD|=Vnx@wVOe$M&1U(~DFXE`myf}9I- zarS!j?DPx^b1uw9VSinyLxx2;7v&Q8wY_j=hQ&D-=YC1vRM=S=mgZcl@A*LtZFYub zIhW;rd0rjMYG>wTSdnu@?pNlj4RbTB%DF1{t8<;1c^TH^T$B5?xh4VgGpx(GF8Ay6 zYNvSB1sOKv+>rZ94^m&4p(AHU?!R0elXqT}VPnpXxg-sLDAXatrktB{;Q^PFI%L?K zb8{}k%Bwe$hpf)9JLhf%o{HVB$*?Eqp4_)bdJfy#4106#%_W`F zlQY(3*q3vkUNI;W`jBCN&i(q3R=LH73xp7D~S> z!?~R2a(_OrM#uwqXSk5_Lhd_yYmT}n!^NBzbAKsUyV{%Ka?Z=SzmjXAZC{3~Ij`n^ zQg0<2`!ih2c`ersvp*}$A;a~Y*K_|eX_oE|X1I~_M(#(aO1}?fxS8{2E)h^Z;c$jq zId7?43vbA9JLm1(x2MXUk7l@&^G@#9$IZjGyE5F(c{dl9sD;*J8Sdr0mrF*VyUvei zxS#WWF2wSd?y$)4Am;-GrV4N}!^4~p)qPzS;8cc3IUnWzao$W+PG@+M^GWW@W~1+! z4Ba`qb3e3Cb7#WY3{P`D&Hb}nm&4Cxc%JjQ%97Te&+sDW3zc)>4jEqNe3|?EQ8i3^ zBQw0p`AR2Wj}yC;;dRc}xxWxK_wQfM@FwS*+>ia@^sp-#-sXIp`^QQ1-wa&M@Gj@O zT)3XnhYas?zR&$vtM$Tou4m}U*^~P(RnPS3=?ouoe#rfmKz+f@3?FlT%zam+A@){= zPdPv3zOzqrhh~P)IX|ncok({w^acCE|8;Yd$lVP6z<%&|jUJM5FGGK@Kl}h_kIwxJ z1Hpmti<+l+JjgHz90Y&g=#Hs2npJx~aj)GW~{=b748Ct{MIyWyfj0Q);-wLW8nY>pS#(-lKxGD7ab%wFvSooJ;oTd3D!#Hpp{N-4y5^poK zfo<@2dsQz)f!<|k2iqaU?TO6)`wZj3@$i$wrY-hlm;g?I|4Ma6Uh_kSiQq&C?YbeZ z{bPnn;3Nf}THy{ECWDh9nW!dZpEFDWr$BhSHjnqUm6gR5v9-m}s#P+z4M)T|z)7S!@C~!Cwrj9oqU}u^HS9vC#jfc4msj7I2H| z)lUeAu-FQ2g*b^_k`zs|*amLXf%!q95Ek3P?K;-~Oelm!C)lY5rSCk`Vh6YbzO3E| zzno>U6WpmEDY`z}Vi&ke=VGq>V2;IZa5wxO=;11JE%t(Yi-WR=A!@Dzk%>xTQ)7N^0}@GV~)nz+W|40r~TYe-(U*5WL97Jgo%dLTboXK@ZZ zr!V;rLLMy6gXbX}dx#zkF(pibwFJfp+nB6v||8_M&0>Czl>)>_xp_$e?w_Dr*Z>T?L z(;i8So8V1IXw;l(hs7=M7X0m|2j6LN2fPD+7u8=0g|N5>-h;o7W+2>c@c?`PzcbXv z&pj3o!H4jVpxbQrT090HtI_B`*w|Wb$yaZpWQR%5RM=f4~ui&?u z>The9E{oUTYY1z3GC_WE%;F9B27Z>)#*E_@Z^5_f()4h=6Bh5lckn%?x*%7aw0IA` zhrgPsp`Nnn0ej#Fgt`&rw8aPT1N^E`x6_}o_y~T~$($vQ_pHSy@Dn7>)_uw6EIxyu z^|g=d{tAn}1^X5P=mPKsi+%iva}(6#jW{{U=D5Ed~}G zSol9w|8&J-P{BcZ$7{M~VKKPi;KF~Sd5*+2iy;Mv6#h=wq#V~Rh87&EfcDVduozZw zm;$;+aMPltU`yf4YJ4Ib!eV&A;Ud4*BPA_H6dWP)Qd?Vd$6{o`k@8BvR$Ft|VpPFV zB7RE{qEM^y+t@n5;!RfuloPu)-Kex~={yi4+3eHnm z|AmFc{DSlKJ0y4bXtAK+f#(@s;zF{E zKdznW=dh&Ul0sDa{}Afnu(aS(y~Ag@!vKe61()d^bh&e&!}5a53;)%sE3cgyx*VuH^N~8v82Md3-8AGQz94dH7<^M@o zgTvv1hYNqC&?@r`hoc3L7Jh%)Y~Y*e&{eRj@O#o`F=Lj)v4Y1G&~0n89gY_~UicG5 zlgi9-I9c$d0-`Q+9ZnTIRru3I^MKuX4rdCU(I5P`b#CT6oGo~^@bA7jY<7Xexq{~k z32!YD&K5eHFL+*G>(A@lEONL|@Iv7)7R@U3VuwowFDdY6!W|qg7rb2fo?f~kW~sxK zf>-oA^<1oF4p$3aE&N_v4bkF*!?l9f6xb{3vBKec!Rv*;QD|jrrNhmFHw%BO&_cp0 zhuZ~jtEKABrPU613f@uRypjipy9MtS{$8Pjz1HD=!TTzIr_RkfhX(~8=vPhAZb^rS z1t03z>Jg0_93B;Xq$8ubL5IWRf{zRTq^M>nd2r}1*j;EME^cj;!_$IK3;(iL^&e_y zHak2k_^j}+oBzmXi^KDR&-IoXI<`8zDELB$<`;!JIJ_+Qvhc48{g)Zr9bOlFUHI-V z{(WMn!<&L{3V*11*wzk*w*}v-EMa`7!@Gj-bfA8`c4n8u`-1Na-%}`u+U@Y6;0GP) zEAml$96lEOsDS<-#Jvum3VzZ%3i;jV@VVe;edp=o6!$yyE!ns9Q(QGjfCCQwO7>IW zhK$QWhyEq|D=g$L5g(zOr4vf4ueY$ zF8z?Q`9G+-9EO%0TI&CU5Un}pFs$UTQU8!(;l4D9?%I)%u za}HxmjxA*XJLMPW9mbU$r$C$7#|4MBl5M5XR0*dS9okE_mwtR%P1PcV!-SF(6wnO+ zvctrZ6BYO_p%4y}N=_>Mp#N>|y2G@R(-fE~UwXq~ zddcaf-x6z8>88Vsk~2y_v#iQGH@6&Sm7GHG0oBdFU&LwI$b< zeqE{j=(WT8lIu$eRuU@SIBY1np_E{yr#8KH=qTAy`i*7PD$ji9u&LyxQi#MTdFFeE z%_TRN{;SmlJuM;k&#|lIF8#{iubml?V|U5jDt|?2M2@B&s^!v)@KR_LvV}Hs0r9V)rPacxvV9A4}KU7waBw!BBak%7R1*V%i zH^XuqDS1SJ(UM2EidL9->HtUQI9KwV$`WM9gUyOL|9LG;7Oox#VRXNj*EYJ;#-jS9B=PQlR{y6^i5#~|-YWg=vf7c>xtW~fPRTo^zgt#&eyGmPlpOa;-cw+= z08?|^FL_^w=JHqS+)T^ypyUGu&dP*M&+)M2!_q%0bz|#{9FI#rR#_<4%p6ZjJ}G^7 zsrzJS<#<~1Y3ZLeM{0JC=Ov$)zN5F+a^~cCQSyb#T7t;&vgAvZzbs55$E%XBO8>fS z9#Ng+P02TED;lqT#|EJtcce|Dn`H&qX;tmi$=yPi3`0G-h%B z|FHL_Pm&~8ewauar6sQJMGb%kunY76*hN(>CbBE~m}y)y)pbl4x-L{^cM}*4YrP^o zGh-^!J;uYMvU6x_CLW?l@fJl7)J(QRdIxLee3og+PL$FHtzmg8(^~kqg%KC@W!1# z41WvX=0Ci3`;Tng`6C*xOMxF zZ`}Fg8)7T_E5bhhPj21*6B~E_#D>@pp#=6H-@5%LH}3q&jl2K$#^(Pe&htOLb^A|k z-1$=*cmJIYs7m|KZr%RV8+ZQn#@&B+WAlyeTfgw1-@5&0HtzhHjl2KehJ+CR7q@Qz z*^N7Y77qLkap1qab^Fh4-1&2G;C~WJ&VP05_MhLl^XE72{)G+cRsPqvZvVxNJAV-l z02{%7bL;kB+PL$V-~evM8{_@7%zp}9jbPu;~|J98l=6eIvfD=@}Jzg{Wmu5{Edyfe{%zD;Q#K{?Z35g=WlJ? z{o5OmT=}QBZvUN)JAY^6?ticWZ`*%=>-OK>xbt^6?*6?ESyc2t+`9euH}3p>H~@yh z|9I>6Z*JWAO*rts0$bFr+y7wW&Og|=`wurZe~bUG_@6~_OoK~$^$`wu>i_nDW`z5n zmPgYB1=#=XR|}3)|FaRvQk?(t)ACRGlV2EIA+kUu^#9clZo-s)=zr>in{<%|D2_k$ zzs@qkqZu9~38Q0g&LWPVVw4;aju}C*|APTd_+Uy?jzdmq9H3x|2NOEj@%;OqKX~=% z!60S9pdv>}n^6BtE%N=l)h{-;&lw#@G2ZfCir5Wn)jrjL40<&1|J2}-j*ww*N+K6#Mrpzt1K(Mpx+|3b41i9f?IS znr+EtYd;EjdKlq2^nYM*lAlB*(6Yqo?RfVG1kc5pd3JO9zh?6LfRY=`xWCok5)}Nl@x&N^7Tlh4H#7bFuQwg3A@nnVx_s?yv4M>w$=b;ojNTr<1 zB+e!g5tr+pg(6Q752i*BVV4fY%fH<|_tLCSoM$cOd%yHsl)$=~UN7b-zK+P`Ewz(h zfn=#_1a}>W-sX17DZ^Ts-&s*+PT6&%AT!~`EEDPv%~Vt$29dHP20}dqmo3FC!YS;w zdMEk7UEQ{c)Uz?6{*UZ5o?(<`4C|GB55@5n3a+=j3w+~W^jB2Lmv>$0cW9^I#~IG> zAkD7irMwYO;1H8=9lGVRLH>mLKQYq{CD9;Zl+!_eD2TI^V>Y59lZ*2`|GidS8>NX=>d_-_r)0>!wQ^o*t;=cshbw;$qUEns zxT3zd((|X<`RnTSt^^LYg z!GOjw4#c{-xjmxE>7M?3%TpiKg8JjWdge0?I88vkuJy!Ny%uX+n}in_g>pj{@_N{W z35P5Ox5%3F&SVSgIB0fXOfXTnS&29B*moc;uz8DH2 z9BX35y{X2&c~ucA^myz&(0?K!S!nanKpS~r%M&`tl!br1Hcg}h+`iUm4^$Wi@>1A6 z(xp{K`Fvu=9~iA}TV7I{U7Y^ec${L>#j!fvcu@I_TNG_u)t21#u0^qBP16ti6G(=3 zt;j0`bn0FoJ44qPVy(p+{o%S-(^(etD0i(bFU^S4uW6kSJ3&Q9XebUeW(UQf}DwH~Q zPzpc>`#9$4Yz+U5DCn2^XKPX}6jJMO&b^7a<>@tp0xNa}poc>vs@_j6p ziBUAeDMzy;eHroT3(V4pl60K`imR9g*B1Atu1PqMK#=@j6CXD(tR7KxYZVX*x)K?*L#JPcY}$ zEQhfy2G-}dy~l&pS9lO{*S^!XNay3cp}(g-`UlP%WpOI4dGyMp`IIKem(D43kxR&z z#tn0y2ASAJ)UG)O43i0bs4*^}gylQ7y#jI@Yqo`&_R=Az42Q6hrlt#9{Yf38l#kG4 zH%TxFU0-bV_MB7DZ_7hcn&JMmv`k<&!J|2SB4noiaD59HNQRrU+e0;_*LyugoFm3l z|I+np^K^u+V!TxnHkx8G#6%c@jE0$vRGhA70`0WfMlsvu9aEa9<gKm^RLcW8{3^M2 zQR#EG0NC>1DdgM}l;Tv4_*(af4lOsgK|xmFs7<2Ghf$1&fJ)hq800$_Ti&@mVgIxz za3RzpFUP4Zo=={x)n?qFLH;;=&thsU6GL!0M<~5+d3~A(Cov*0a+l}JrVXG{Zc(tF zeb3xgktpp8oJODEgI6f%?bS(iI0J$0#!NYY00MV}>=Lp8#c)-g_Q|?S)Tm00U%nRD=%viU zTO@ByGy3Bw7VG6(tFndjN`b=4Z-GtFwy1*2AxRtFER3^`FuZKPU2f;&980QOfru zbU$2ODv{+Qg`9NDOK5V)F#ZIaT8~|ie}4LJqze+j)uHj0mqcV3)46}&u%bl_&ey8j zby2k=?_*voF6lPOZlW~eg*Dc^m&u{Zq$tL~dJGnhxPxiSLZ&i1>gScoUz7zY4Z5Iy zdP56Mg&nM3*$dTRXW8Xd=E{h!$kobFD7ZFbKSxteCXWZzbiCMqVHS6`wmbXtkJie9lT;GUEzPV}=vFyII zjS+AZB8~=WFvT+@3$6PFdyORmS z69mzs*Ki6NfNL3**YGz^lhY>b+Zr^jVc#ybrk@+nxLDcF8NBSq2Ecg62MJR-uulsi zL-=!CffktO5EI1sDVpJ$)6gexC`8r)C3Xs(BR;hav#H^;EahhjCWlcB0ZT_QrNEyI z*RLTutS%48lhI*IoN$Y7(v4Spk7bgpQ)DHe&=8{hB;}!w$sdqFSN0mkb2$MNg>zGuPO5A_0St zXE;$Ss!VgrDtB?bhREra7E$wE>eX8)|I=QXpgfwDr5rr(^U^7;XQL^lq%VsyTIYO1 zMQs@@u(lNn82(Hwaq{$!NCa@aeZ&zY$kKH{LGoLt*7S@1d6X>WyJ1*Ta}5Y{ttL;o zm_=y%PP5TmfFhv0698Nx4qo_M-U(vq6vbWUd#3|Ym8|4XXsf@XR?9bAzD^iU5W^?w z#P?bqfS}|U$s%3xdsnnm1)C7iWHF*^#1rMP4$BZ-u3iP}AXB7^9OJP@MRSz5r8(KG zD8a8}?7%dN+AS`oC|r-E^DK@{H;?5C6eSbbE)FD04q!I&S(J15AFs3KMcp0wPiyhc z@9cR(?X^gT7@aQ3(JbPHK>Yfy4k9RCd?NQMOKg#Liln3_x2P*H*o-)t%T(c$Y+Peq zF&VrYhAnBCS}anmSTat&PlMDQMQ~L$h%z!$JHr}CB&r5Q*%C5*a2w1x#AMR=2#gA` zKvYaR|HXRkfJ`WK+N99I^^J0Vi)Kme#dv~(MKQ9y zz`IqVu#xUR@cgRefs}kX!)c0i{=8i)X;9^QoFzP>BrUz3?Le8;ib6G z+SDw@eznE4V>r0!2>b@=F6VfbNLq>;Vf3|AbySVS3;BB$FK9~()JM%DKE1%4EsTv( zHIIMg`Ys1F&SvBg&7ycwti}>6um|7e>&4I3_maz5D=V|JgVuZo#RF@=7m%0BkZL87 zFty}5R|!j)RnJM>-lKe2ez>wG#|#;KKz z67da`7B9;SUnloX3gyf-@56yp21#X)#v_KJI3g2+JbtG@x2xGm%A<7Kn?`Y1A=7Y_ z2k(X*z0uO?37fa57qxrrmV|*q1~r{ENLq~n!J>SuhMq|fOR)+ie0Sw8NAMi>w|rmQ z6SGBpQ46~}m1aYAc%UUq1AO(iiz@nTc?f`3prujKd-HgkL_}U(_BNIFyA-eXp4*wJb`aK|p6SN<7JLb+qrvk{G=>{iq*r zrxZSj;Jm(?JK-Otlz1sTgep~R%2QL1!&vnm``Yb86+?$qWNW>M`YTh*Uva4-;L z3x9Vk_Ewoe_*Umn^^N#K^;bHtwVvD^ETwC#)KXP1Ym~@v5~HBd@#Xqf=txz)Il(s? zRL$iFaqqndopU*Zv($heU+H$rpNxw^>{lZsGDgsl56X;N%L}-98SsJZW1VLs3qgt`B|};-LP-o2#QSZ#eYdaT4$( z8&`bf_K{4X=HD}gxb;p)K zm&$y7?p&7ANuTCcx6#6M_NnoVdk2hBHufJ_6c%AIBwg-}R()gOv5W#wk}L1g5QpAy z8jbr94=9-fN$h+XfY24}*Q#ju0?+6TK9vH_Q5a%Xg4~~$fo4o5F*}%)wHmM%4&?|L ztXsVYd7IlOQJO|%GSVPnDPc>A=%AS{L<}k8dO&X;4BcnJ;Ect*XH1x-!q(3?_CK`9 z^l`ByBa>}_We;qQF7Z7n z)xPA2D;$vttYclN+-25d=;69VeQc8sUS=Pbj;;-6sfi<4O0m)86wlS$UF8b?^>rI) zY7sJ}3*JW@H_`=vu=dAFSRWCJqff96(y0)Biswgd(w&P5QU&ybr%ybpwnThOl4Ls( zL#q6eIqmK6Z~m$Ph=-C>j;S|?HS^aGfV=Uy2tFaaaVPGT|jT7o8P*x6cI-<#`E;;ZI%<2?&Gd{(HM}gQ>8_E?7%NfGa_Uv`kd8FG!x86^cs^6^30VJ$|EazTl4_T zKr_FT438O3r*hi%MkNH#_v-lNTI8g;?b4Q~knJXslkS#JtGZ$5wPRt?HOaM?ifSxv zJY}g-Qx^ujt$E53iMSQ&v=-+qgMzK%rKipDQVT-L$vX~%fl{#cUbc-q#q)E-QXIaF`1ClMMP`(?t506Z zH{wMt-cN;^7DH>*k8ALe3zaNgc&U&Cf#z#QN0`yu+Fz}-Z7=uS+at2vv~0WNz%zw-0Fy=ycX%Ez8TIKoAHM@}p0rkm zZY0Ba^;w#?GX|T}UzV%`2tJy$q3b5o)&j*vGi1+dud9P9z5ZfgO{-Nbt+S%A%$H!e zk+&-ggm+S)(JC3uq8PD=FT^^zM`sC5BaVmJY#306*&14@=QF?y0pdG5wMfDSWj4UB zhK+$`LOEcl-P|57NPxqJc;@?<_d$8^{7&{9BY8ONp2%ehq6Y}xDv5Z zdxy3LqG;u8bhIgGXzR8chW=L+ld!nLds7q?XZ)tTISpviRc z_)&jAnHFUTqU>U>7}4a%Vu#Js3Tq8_)wwGaZG~ZydpT)mLOQwzJ{O(1Yv3IH$vWPa zX3EiC#X415P-j4NJ4&ysF;dao=HOKlBOy2dRv4m96D`AtOt0FpDue!JI`wTJ)CsY5$eEGTA_UR#Y*T zl?rvGXQ1m)%G>h5d{*YNMf@ozxBz%fdJTC5C8h`0Lq#;SEr9E5QBl~coDDH)SwMc1 zw>;Ue0qX`PUriS&v_pBa;208xirF&1abe+ za+Xijhs!u3*{e%53ng;nGg|Ihf{FjGo9V<*=#ggj0) zlU&g+Qage@uw2Uf0>PIxAIzECPlC^u?AM%gGD{Js0UXWolBRMMx9B4ecqmYYEff1 z3_F^HE_dWsmXHbCW~G&JxyNO z`;T+&cXMF4)U@S|dbs|5M(3-s+YNg{Mj}cor8cG`X%%#6#(g}&{JEGBhvJmbRg4QB zrARvo8O+XBf{J2iR!RW8l(V(gt9<3OIv!lw%mVRS|anl1^w9+zfvqp zBdUC^@7h*~Pw^{|)7oQwDW?g_>_>rEf?QKUx|yv^*1M*@N<}BNYT4DyqRY03YG_ew zXYCZ#N-=}dE{Ks{#u4W+#w3g+v9-|~Re8X*YW5?!;Z8Qnqt@X65}k6M`$9+3e`qsNFLNF$~F;LyDm?Pwc5RL#&S;4 z(I_j-r~EA&(at{M`N}5)n7YX9ms^KG3#9fozuguMl=4GYt~T+h}F@YD$t8X*r#c^zmU(i6UCQg zOOpItGA-xqU^I0^b9OM6`H^nnch+%)%}Rci0Y?`1Mpd_c;fRt z^WbV2u{wv4Q*2dF-PNu?U!NzK@6$j63{NmeyCi&0N<43_OESZ9NjBOfZpy9w%`EB{ zVu(~*F&1@&Q}pVtN(2f{g%_2X-PuK?cUYJXE+xvdok*P=;^Gn-d} z7?;6JaU&~mh+fh4B0ze&6nu1E&zF8+?MuqGGx?3pisfJ^z~Rp`3zP8%2F<5~{SzHtukZ4Y_NDS#JFT+my*{%Y9!= zJ7=^ueE|V>sI|7n$nOmc)q{CMTV9Hi9bHK#6oRj0`lrf0Z8XJy?5HOVzJ-dUuBGa}!Cy zdm;)Yv#I~CYYn^@EmSrM;1@Pcefm$Ut_R*Jo*QEs)R@5yiev7Mh0Ovi8& z;dRl835N!iW<59gBEjk4n$S5JAhJlVig6=qQ_r!~_hU)2_`m8R`mU8mo7=l&u~kv5 zJr1mqY`KZ4$cfyjBAdfXCWcepL_g#>doK`WL6&K7{N>m1H4l{;hLp!dP0g1{8 zOOrTui-3w@v?}DgD~Fg6(k=N=vX9>tX6bQsgFERfmRE_onYA0o>aBDm`D{-KxOJB( zk$j?wsXad1#pYt1s!0%Pdv{7>-wI3l>L5!cGS+DWu zh)oVH7a7|FrP6^!8dMaAq9Tl|x% zNoJae$s_5OWSotJ_DMnCQcxGCuHzwh6giQMfA*Af*z^Eu_xIpO$Z@bC#t@ZgE08?bTm5mf4Z(q zJ>)D(WOoR8HSbqwKPU99U>Xg?e>V@k&23QQgl2h+)g+Hx5%S*6LvP2sr!Vi;s>vA5 zxIE{TKP@hg0o7(}AM}tCycc6cvZR9>R3d0Fn;kOz@hN4qV;a;s$Ep!$Ij;&d3bSU9 zl5sTgzuhtB6wk#RvbNP3G*Zb)00-uIj7S0Lk6#1hru^qJi4nQvXj0Xdyb3us7Qb_h7dbHm=6(X`b>TVp{bg4|P z1~)tUtvDb&;<8&o4S8ZuGAQI02}O%a{fb6;I>0buTL=J4Ey8L3*!LyGl4q2HP>1;B z5|wU1VXG-!^K?X|&VV$-%(N@isww1d1CsR&!sG39cl$wsT7sgR7%Fs;t0*_rwOg6XEJoTp}t)Z!oww$!=deF}u39rir3=1Ev!0<9f zWYG%75Djl7x8!}Bz~S#Zchk(VW?*(ipWqRir2Y8Kawr*y>e+;}5Cv*R*t&Ca7xEu8 zbi_CoFn|K~(zB~`xdLT3jz`r00>P3$!+R=QJ?#OK*p!?{j~-0(Mp1WjiSX>J4m^-7 zkW=_yY5av`cWwSoCPWTgB2+&bo}HdwIbhO9w&ugew0!oZ#-R_aWneO0A|ERd~hN|fdu*K$Lo*$fn3 zx=dK+lma#^owW!Ik>NIIAaF#2I7_1&EZi?Q9J{0@u)toyfGtn$4eFdu>D+&4f{vun z-xZYCWgoS1Ti zCVi7DwOD8{#WQ5t+tWnN+#9iA8IV?Qu-2CGAJ7}jcRFv0UWj?Sd0qF#}~ zXr?!3q}8;|b+gDsOwm#Td-Dm3j#&wyMdeL`1t0hhmn>H*;f86n|TP99?Yh0wqd zO4!PN0UOCJ?OLKZo}e%^rkT~wV!~(u+^|{@!EZjc@9|Z@hX1l%&}Vt3HFma(#i~NNjA+M~Cm20-GC65ZLP=FVIovYsW*b7~6Z< zAIq^JBQ!*%B3zcC-?JzNkl`kab}EwOoM9F6Ev{(ygZJmcbO zD3GB>$VX)2zahUG&-j*?pimusVE8DGXM8ZGY=$60eXAsEJmW)uLmo&H`FCYTaWG*K z;nKqVeC_Bio0N}*^jsVrI~xtl_PkQ38EsUy{}@ye=u-WTj+_nEK$<7>J&RCAjoolR zVmRQAf#vd+>Y83^N_9*ym$57hSa|h*KttGrR|z{~GB>(j^A+GnOf!-Lx4Fgp)XW8w zlYBd-uY_YIWGK(Wv8rE75t=|6tUX9KmxSBE9vTI8?ya^?n*`5?aQTLKU442}kZ0jc zw1g_#*~EXczJ@q;GV$M$AU^eeL?hzgH(VFR2>71^gL->69mu^tFM$GF3w4%!^RODe*q~hG?PUrOMEszXL!KyJvhy=?3SuayrB}a?Ls7WTU^L|Eu0WZ7l40`Z{`k#jdU>wC57J-k1xLupILl1?&@-`MV zv7qJmoy7`?j`VuEF>IMx(-OsGyv!+=$)cjWdH0qtE9~Fjvyyd^zv(d8*?P(mH3c|l z<0>0R)H}s$oo)mQUOF~RdE;s>QKKa*0%y$gJUS&EJKBQXg62YGD0DK7{|ujJmE z%J^&?qa?+~agmCkLB7X-t$hL#AntEezn876YX`wRT+#l13A#3f7`S^Q^Sv0QDTD`W zbn><`Y?B~WYh6(;n7e>W?^LQKq#6_J%70o*5ldmce3A3AME`3IYw^pHf9kaaO%=qR ziQ5jrAoY9KfT@;O2o@Vf;eg7Lq7%%gG)$XP!=RP;UrZyv=RK&ZFQOv-~2%hmI1iZjBlyDbEMxZ7bBfLMhlG z!aY^Tz?@WDi&VEpIN+e5ke~l06#J2rVlww5OYin50U9Rk|72CEXD)QYP=Lo-oX#T- ziK$hhqrT;-EmPMu0g}|#jzED{GlxrNP1@=`y64ThoXc5^S{;eMRJZg!6vtO6xE`uf zVyp-i+R5h)uc4k}5U3e0(|n4$Lt+SDI!9ax_aIKXOTvRRKnbojbX^OCDN!@on|Y0d z)#@G_p<>FBX6Y8ZXFE)RRvrGK$8i>6C?pTD|);%;!L1* ztZWzS8&v`YupDW=%bp1*C`KBm_)_DkCe;7izpZDn#^$!TpJH<^O1&h1eeIcP>E0&L#P9}hGJBANi-YJ^l zuoeim4(&;)T6k?OFw^LL%%w`Tg~?Ty6mu?C^A!Z+Z3N*c3^k_6|BT+*%582dRC(3;Z17<$V7J4FOtKg? zHWjrHBcHGauf-k#tPGr|1_n+8ujj#d90khkS4PhFtrO`iBWJ3Kqfr`Or_(P=;|w}R zb`vw53*%s;4hRwlo2i>EM58dQU?`k~?1tef2_TLIq3K`M*0 zRbh787y1LnP>MeeF+g2@srmvDUDzR=*K(;$jkbc5tl>rT7$kPcX?G?*hcq{eRed765g+mb$!VHAcD z^B-DuKPr-jB~iMY(JdWyAJOMPP4SKDaxNWEU8*OIiBhY~a{Z0)$Ov!iDqa{|3B#N( zyb=A;@a)t%tXmVOr9$h1_8{hkis|P;OcaKDQ$z@kKN&F;;DZ}+4zA%#oiZqgFIzg$ zH>)Efz;f$sFs&;IkHHL)4Df7POm)2Lm95Fw)5w5%3+}tP9eLK|fCu3T4AttXh_TeJ zqXcagwtQ)>N~-46M#;_XWi`P*(JXhVf3F=YUoh}hMr<05SdhT~j>#`k#wq;Tq?EDN z+*Y>$#pe=A*hK-N{o9i$O`)OgD}~6k;FMY}MiRE{mMn{0LqBw=gFH(LKAvy2Zuwv# zsO{mg3DiCep8_-E&Z8|4)IPdFu|eO2M$%l_XTYT6!7{|Z!M)mQtr(e=;Jk+ncNx2@ z3PolEDF`C8zt%CMO{$zEq1Wr<|Li?|Ma)C4uD3; zQ7|lxD?+Wo{d+ik=beZ5(0J$3c+CSMS^#`;V_eA3q2mgjbKkt4ELDx%c?$ z;X@Q$h5pWCJPsZ|@Pm5~!bjo#oqPCgbcOMK^r1W%=cKSA`-9bOi~`A4^kCJam#>Ck zXMhFm`@{Y$zQx8&X>3D2-?e_|c#|^pT^W3eGJFji%{#}tmg46k>#gWrNjq|Zc_BMQ zxp7$QY|{tXTI~|&N(Qp5sZ}d|u$p5!R|4f~$Z2xAtCB^Oz0YWtYhplQusih65bkVE;2kJK0@s4C|!n zfZ-Xv!6=R|Wr53WntJlsZY=o~KT|#ee_iB%?U+ww3CC;)Ii6%Gzc>-IJ3I0XFSr5Q$cx^_K4W=S?$w<}&Q`cbhr{~!0yTPwblJ;_)fWA^g|_UlPtrx@}9@^Ytr ziYi+3*rn#m|KR6{hX1cW5X0&N@oCi(-KG|;yTiF{SypwosMci(c{yy!5(QO@hH6}& zhTpm}K8lS4Vgi*-$dRa~Cd_s((_(ZjCG=nFpRuX(Uxl@ZPe-ZW{un7tuu+j}@#M(5 zR7nCI#dxWY)XG>@jEH`ub)M-#r`BTn;i#xBhrrv2t~l5Fs0)CwZ*`9$c%FkDe$7C< zct?SvsXZCBxxF6&T@(_RESKKZWoWscxDZkEjZN-dAx7NyT#}OD*xp{JKV(%}4Yl~W z@r)ON24f4deX$xYNB|wjdenPL>9PaerFe8NC7||FB}^=!md#4euEU7gggskqc_HQ; zktjGCAH0fE?ic{ki*lK|IKM+h1(GZn0jz;r0?e*vrMpoWY)Xz#Was%dkj%9=XPR=J z_&OjT#57fX1=|qw22yAO*GdwGrcDI@hJu*9+K-LW(2(yuAiG{68 z*CqB8%ez#>k@AkR%T=HM$8Y@?z+0(+65OoQ1`)Q_WNwzM@+p+@_`meOw6D$F{}<|t zYT)ilmQo4O5$28x73GF}d(-k&0Q;I?`3V~Th#joi2x(SH{LgB@xfsyiqfZ6I4=I%> z85a&!m6k9Y#r)!K9^>tP7Nxk%T5$jJRqI z4O*%b@3^Ai3O&C;4ARUN0j7M_hQ(Ua0-cQ&JCH2PapBL(d4;r)&6T#a_AuV;{^cp2 z8?h^P5RNEG6Y75}kQ{p6=5~Ncfa5*>3h+q@nh1Lpju}UTl`#FUnBdF$^UH`&_c03N zh~PbxrPzO?=_`pi)koFB;@InOvu)XDTyHQm!9@46 zaEtUd@i17@3N6=+D+rofi~nlvISA@U`IDEPm%(~)$=nRjoHf10Jy`8VKf^dVJw_a- zd>_YH7z7&SyGLiUs0&DDwy%duY_Cq^LCGQ4fKlnB8#jVjMW%5~gKG#v8kQH;ZbkX| z?^q4;3y%{TOkc{TNvT|U+)EVYl@91^mXRpPn-BMEL$OSC{GLgLM6H-Ks&F4#^wgTm zYvXNhABan(u2GMk{UxEf@Co&Q)*Ce;c1a0vU?X}S#W+jy_6U{>PqnsAS>^jtdhMfi zm(!cDQCSWo5vA4^CkzPF(HPsJsUI zhBArDGpROe7_3PwE999nuvyS%`d_G0Umg*TCzzGFE=x!I0phzD;l5`Pe~;=Q(FQ`1 z7I!O=%benwSBoA118a3HeS9u8$z?`TG{!wTV72s`b4>r#c%1yth+8}a&eE9pkGqLs z3iO@WQzsM5Mv@yQCLJI=Gjf>x>Pf*jzPMx!?Bo@q8Zq}iPWaS6>N&lZIH9X9LNhy# z)S|0#I4D9X*?F2hM_4 zV4P~Cl=7S{EN$?LQv9*LnzHAQ{Td|4IkGSoHDiS+=UTv=c(jn^UI9YzT!=ZP92$9j zK_(ds+e5a!duvrGRs#oJV}B0@$A~ zd!Xu=BeVDt{iO4{w9G&0T=CpUK4)lV-S=p+7*T(xvlK3kKEa_h z^3ExZx4iR=J9{*x^0b6fD1mKn-6W9=m@Q9lX_|L1-B%EmzuKZof-#e=`DEv)J;$>b za-^^ctu;q$*r65A%!{2ZNoUm3PNTtrQ#G3XO!5lE-NHV|bD1v?TS{ksKIJT+kMmxUF_Ccz0}*ZaaYF-?5t-Hazd+VjOi)RT!2_L8#e+7@W#?x&nN))DGA zc$w+aEj!zo5SNedq4VQA!F;%s*;H!4Bc7_v-xb4RKKIYcDr5Iyc|2@Pc{Qph;w>X)~9-B@h zMzg_$CTYf_cyM!n5Ju^>mk|s!!G|eFS4K(CWl8Vd-1qzk-b0WG|H~<2?i;A^JVg@> zK87+dHcbZ^rNd41wX7;{uaJBn^FxO5FhPNPZ`(YX2`p<-dM(8{1)!9_1OT8a$cBNJv!O_wEhwjjEmdTJsD9&n2VKX7l$2w`%^&@4IQDGA`d* zH4o&PEgmn;h%Is6+A{9RyQ+i^NH)8Y=9)#bA+6C-S)tw-CH=TR>`G|IIE4n1DJJ1F zjFa6sy1~GC@Wef>4e$M0&}enCzBUG_Mtdia`1mI(yPag*Ht>}fv~gBy-|lv2%9YD1 zDDT=mZGzYpnqcSxuR2RWqq3Z!p$DOHb+Umpt{rO=n~|I=0okxFo}8d$H=d(~F_5=* znQ9#gw^zczg_2+d_eNd`N$@{icD1q*yPRC8JDehh!w%*gOBL$alGTBWYB%{xe7~+c zA}C&@QL5UNV>UZvbao+3x{eUlsx#112f|ts`WuFu8l!MbQ&C6b zKzjyN_ptiq*C(R<_vR)Sd6ROyga0kpA~F zj!5=Ob){C^lR|#kUp9@6&~zY%=1$6I%CexVD`;xLvLZ&Wg=Hl{d@3t4-8OHv%}F$_ zBym+V&5r182>+Exaedq5dI{FjL0Y`NCaX)F#qV@SQO3}cDQ{;F#QOUjy2_huL>;`= z;}q9=%G;>|eI=DD-&sAf$OHM`taH$-sE{mSqE{c`fEx!yt7Q06k~o=fbWQ>!*Z!|| z{wBv;_+P62?s@*jyWOFH-0%$OY(|OqKIRAHrW{LXrv){N5Y#5n^8hQXm1aOC#Ar~~ z?fcy#W_ad}BS{B!f@cvK)ADGO{>dF@HpAjwnaOpv-ZrB}n->w|8H!KvjIu>pm+m>2 z$J!=m8j;valx2yGT8}Cu89iv=YO9YH>i{Zk^)CYPp~&|fLhk5@@HT{ZwaFM*aWdPZ zi%e~rWVClLgPSYUN0*_J3>^AJ-^HtiZg_EllNbduT>_u1d*CuTPR#JtVuoe8-|i0R z5Zw^9=8`o1@Ia>^SN#^^K0C?^S;7Ay41JxJGqE>2DiZTtV_$YK%6P$I@&d)3%)pA( zH^gI<#k|tG=A6=aMCovb;@F@iSH1I15wky8jS`aoKuLaMEiID3^^rzzvstTt9ZhhPQ zPUz(^tXL7*e4mVPn7<*B|G7RjeN zEPf9R%pp8VTi!XNA3?hf!y49QZuI)&S(0f%#M>qlJYHa?8wQAMc~Wniq)_$z&$D#8 zPv_(em>%**PCbK5Rqd&gs=CKIYPy>w@!}$jacY~HF9P%RG`z099id4c;I+V3R~009 zepD*4s8Oypq{cEbj!2YFad=8YoNjrXCdVj+4Cp7u{DE!o4b=_Df`xDyn=LE@KxVtv z;xu&DUukt0Nf2#p{y3u?Id{Zf77Q%KD!(G0$$#{73f6yN7&mjer0-h4UMLKQ*?1hi z>MUjpWM8Ty)sqV7w*d?c1Z>}K9jQ8eD1r=o>ongraxOr)Q7wLe)K%s2(=vx5xN*Ay z&rsainK`T^N`OU;;3LAR|0&EEWhs!pyloogf+6Z3*}B~2LAZ(tVvBvmQ4Z9%Fg5F2 zd&*g@-mO0jLvv?XW@1$7#~I~OM@y$ zWK74-mC(y7#B?%Ip%=$;@4_39@6F1ZH6Vk{*%xDEtkT5#Ke2TMFL0K-P$kG?yjW1q`_fa;%FL0pZY$lQYHsRv5{CnR0 z@((6Le4T^DM7zyxS&wpp5=d;FG09;gYhPYQe0q$oaI9J?K3}uiH?nK$$XaQs-U}Q+ z$gNq~p|wt!Id=UByD|AR@vv2J24T5=$1T zdM9^KQI%0elt^XQz}R0Uo+)&8H+o8lF{N*9q3xH?&1=2B*#;<|p~V&6qa;Aw|E0Dc zC0RQ4zuxw3tejqxzSE6FcThm@NF#HRk)202o^Brbm-!corIhaJdzNokMB<3x@;l{5 z7}r!X?&@_nRE$3`WHVAIe%K=(G|_0TDA16WnJUYaB|VB3$)4X?%X(SJvV%}AOKIzd zD$5$-cPS>}1x^!6QsWMZZ6Vbn&Q#>;zusY;@6lw@+3CEIFuv{u3Om(~bv|j3(A6=Z zoFb^jYC9XDs$$ zQ@84~m9ZE`ip!*>oP(4f7_|Szs#UKCuStwJ*3Em@(w10@FXoxhj=F*dy^h3@gkkP~ zRT5EbZfjh=dV!%c+2<>#E?IHk(8o5npI;nrdGhEl6(MAW(*!W(S%L}T{u5hnjk1t| zgi<SN;RuhotD^Ls^$DE8vuzdK#>n(c1W4Lk$Ym(d7e1e z#i-u?dCCoPaS=`{riQ86FPT`S*4L9|g+0!-inF|~?qw{c%!X!2zU{CoJi`m`L>Z^Y z>nV2R=nh!4lkg z2IQI{-cr0${{7Z1Q2|R28J(SFd_p5K(e)rUb#O7A3zG_mS#r)I$|An_v3L}=0NYoi z{dVhi9}3}@iIRQ^PnvkVke!;#y(U) zW?3^c6B2ZYsZ0qX*p^l42#qjAYNf; ze92If2$e$|T;Q>CIo`2}(CcIvprcxd`YU`~!OclJY5k@6ewT!$0@LQUT=3~qc{Z0Q ziG+u?k2sP(^|7_XTb3msh#H*;-*g);tsE| z^inNWMd{@VYrLvXSrJRWw+b8e3E3sp{~f(|#RkolT%NBw%@ARGI&X|u#oHunG4gU3S2E-!iB+1Ur_R)#r+7Y8 z0Wffl*Cpwpe_IDJG7?AR8gc)#`pCsBjF^9mrrrfSn92(O9r*$OZL_Xkva?p%%G|RV zcpRl%gHOsrzSf*Ce6COAi!@cQ)UG5vP$#guNL$gHyx1`!ZR#*IB{c=Ryx9?3vPs4F zJ@blhO~T^bplc=RnB}@rn4F#MdVZ}VAP(Jdqj?4PbnrhND8UfN}O7r8vUhW`2I20>QHv9brC#v64bsI}6S54g9} z!4JM_aK+X(L!-g$&#aD*ItS4c!biGa`zAF0g33Yfs42}=%9 zQvg-jZN{{Mv^fNn&935zOh(ZRzl=yo=joPA*2uq{4Y+cCW~*RnksTnq%D^x%#K{7% z_c1@F^P})px=TWZNd0;{hvr%GKIZ4rJ3Nb6g3>4$OlX)6W++JqoMAje z38Wk&KJBPwX_EuV+jqR3io9Izg!;eU=a!bp8z_69(X3jE07p#ktN9G#{WPEo+1x&f z(o{Tg_!Wm+It1c`El)A`cO}nr8AHl=KBBAoMU4=@msR0v%+oA_$OZ`wWMP<4^>M`FH+4Vb(*@3K|@FK#8 z90k`hNArsekHwScB`CN)7z+>ki~0}{!2$jdj(9xo22&hn zv1|$~cN+83G~u#}yt2eI1vC*Dn&Ja+gm7AG2=1^xO1WD65_#^@R*=9wcvRCSoo-+(wl07?$RihFX)>ZEl^tmUQaN~5eHlw(^AE*-HBM`E za>j~$(!xgl?K;nG8X!V+)hR${VM90WI$9;&Rno*vzg(S3Y?Sy3oZ zLbcHLa+Tn|j{I>=JGkr{_RZE<#B&)%b_D6;>r7gH#HEaz5D*SuE4W1%a{)a;!)jAG_c%fUC(i zb=R1v--qs9!olZL^>qJYx*&n@?NGAiN!!&oZaP4Mfr`gjTChvl3cxWqQ%|;Jb8DR+ zOpv5quW&h%bBgECDxzg|lJz9{k9M7_SFWHR#f0|gx}I14fGTcYf|gNj*_=180K%kl zh7}I@pVM4Q>RI~*{kJ=VB`l#i>*AIME4unVg})NeWU(8^BjeU~T_GbCD$<_WmGg5= zlhbpVC!*{G5d=-dC{fI^V9ly)>l)~qNgX9MnWo~I&4b^$sy~Mun!_hOlV9%@(z~cEI!k92O6j6!sbQ!k zUXD}URLfZ>g-1Cx82=38)q{=F~HEjn5e!L+ax2yy#jz%a0zBh!>r=XR`hc2s2mrIVX7jq-yD^ zer%P8)1v&SCe0@NTeUF&>3#4jVSO4cEfZsNG`So@H#bXruBx4EUxy~DF`VJPzpPIF zGG>I~l*Tu>kXkfaGc?IN#K+Mr;)Ouh^VR~joKHm*@H)G3>v|M3i^S>5! zuZs1QiUJ|L(@}}uowZS+qoQ1|4K-yEV^X$X*}fFR-ogP*VJ8_(=v#1P7KLGq=ZN9K zjmo_RIN_9~1BL^7gW1A+P7qtjlRYYapyp`lKQ64My(x+a46dpb^it~v#Zica0-btg z2D`P%+lTvk&8%tswK_wo&ex&@sk+FW;`sX{kA_s@~cmKT-!!b(-XRuNM=Xw*l9JSOEhQ@V zeX}RA`VcByWFEO%0U67?ezF%kTQXs}%VlMCs^~dJbhLNw+$bS``@qXlU{A;WOohOIW^zaw{$dBR z;4QXb-)L)sHlm55I8o#q#@x9zruQ2PoFi{;d?5p-Cb6 z8?69?8A{ZOZkQA4nw`Oda#CO-_A*%nHUlZbs+Q$TxZ;Md5|Il?O9Jd@&vm6;sSj4~@FwtA9V@HTvH@SC zK!WE7ADd!5TBU$(w?~Pr_#9ECoO?g4dL@6)+7NjEG$OLHCt`B-5uK8hl0g6!@xnoX z;(y5yg$$grH`D9Io9NBt&6$(|xP5Qut$Ut-&-d@WbMLW#|Nf(gkKPJ{w;$bmJ9zv6 zJ$_qFS8MHV0a^Z0HcNz0)K+^ZL;rgyj?WTtSDf*E`XWkzpDg;*71NH^w_IGs1<*0b zUSwbG$+L*@g-)M2>blHBYKe?m1sl?7$lZgt91{RjxMF*Jli4F4Y~a&P?`PvgE0&j z^YkeT&BShemS}s5WkkJgJ>%Tfi3TuDOJP!83&j@+P6yY7&Ph=smRy0scGaUcszq+8 z3@y2W_C%{uy*sPGW6=PoU28hI*As+$F1)WVE>-7Cuk1ku zaj#(p{$QFQ*elgO0OdC{)SU_rjPUyebHMLE_U@IXD%pVeF`T>zEgk2LqHKmr4h9f# zffMGxWjb00CcWmL3$vS{nP#vMo<98DRS(~JdB$S2q(50Tq1dV9WbL#cD!m~qjVu6? zE2_`8yd>jo4^G=PQSaY9(+eMr3Px2zd@Uz-mk0QiRH=+_+jObc*LHN#vg49#P07 zx5X4@GnBkFq238fF7PCxq+=29lpyPKAHbjDvVCLqgiYKJW;NmJkF&A=!MZEMFLfzA zcxa~j#dv~(#Y=_~rCL+N<(9DruV9mgT5UD@g?~2oU##uaP!e5YISZ^ZD=ODfh!XCV zRk5fbZ!Tg7k;11Ci%*LZus!83fW$h^N=o!k&|cu+rlUm@3{GL)VK}(ShaXd#6eS18 zDM#Gjps5#&Pp34d6Sa+fwsH{45ah4-SFK>O$Ulh@@x~DekE0trj6T8XE(s6fXc9q& z+O2w<;Q4t`dF{%LSc#}q>*g%;Ue&6Dp|&UT@`EierUByEwA9XAE+ilJC68e5WRQvp z+?JgH0!oNt_T8}~%oZ=huar6i2uV6Wd4+=O38NVaL9%Ot5q8v+i7Mw5YC8wEf6N?UJPnmDF+s@;t~C&LS=<}-5c z;J@lFy;%~>(!nQ~!OC+?Clkz|1+(kB-BHHhhP`M)WBOl_4S+Vc%RG6?HT=~M(>AB3 zJKA4IHcKMJwmey6q`bRvXD&mLQ^}HW_&k)%#X@!CftuP%j$75PY9v-E7`_FpSWgya zjgI_^EjOWW1=DCC{=0byr2#bKSl7uuF^LP7CA@d@5YVZm=$~PEgMO&mdy;77WMkLt zmE1G|t`br0AGYG6#aYX+1k_pRY_%m_d_xp=%y5arI9#I#43+)jWsrP$);JhB!#Eu<9A3pJ zxE6njWsPBI62py>5-n_u%aW_p)M6!F+>ol`Tqv{-Gi&5g5$xa6 zPH8}s#S0V{L~d|;GRp7_FZ{1p0|+yer~rZ@Of_oQi4z)3UrN^3ya)6U@hBZfIMiSG zcUx?OSvpbVwzp|2J0V4{_9K9FSYbf}xzg#-^u(UXqFDUF_TNKsENp7%Z@!=K%EN6) zQ?f^s#nFwy$Yn^ZOnY^6Q~=jRqN!=%<~#M&piDYvCWMF$gUtwBb}QXeZEh~X^QWWH zxk^cZCGe=N&%zRR_X-|}L&`2NXNzMRT;tIHMxgZ=#8E09pW`UyTV6y`%p?}W|AC=v zRuv`XO{jmjWs37GogOkucqNZCMd>sOC`(j$|E-v1gEX2YF^*X#246gvKxPG3H8R^}7$ z4aYf-W)b(lUJ3=|knS?|O^r*6;Q3JUQ2Ae}euJP>+z|<3-F0HPieRvKqTG5kwR0_5 zF~i4{CJXHu=M?)eMZtmPDo-Xu+fr*P12xUY;~63{4b@edX_DrYvcy$&ZTnk#YIaH3 z8^zV~YW{4UMRB@(2__Qf{-sJ+d-bS4zc{L_p{or}P5o`eqd%8j^(n)$KXj_fKeoG* ziO=w2PMK;c-4)0<#d96YXkk{bV43pQ^(l>a7;3rY zqP`BKw0>O98U09wy?(YvFIr;%4ql1(JUrrf=6|K3bt>fi2&r)o*5BL`$m5>L1`%Qu z#u1S*|IYwtmF#X68pDNvt}37;6p#5=TSB`KJ8b(BqP=g_qiN$Pj{Pq;#@K*|;!e|7 zU_t&$n}fmBFQfpgrWln)G+|h`nLr%1^}JdlDn7+?|IL=)mVV+|3BN>J{LSvU4j7}X zt%9uSDyTjAfUycJtl3@`XYx64x8Wr}%t+9ac(w_c=~l72jLYm@_ElbTtR5_Qd-Y3B z0mP<&I*kqPS1@3Ud(&K&0(A`Hyyb?lOTr5X7|8SQ7fOGPkXhe9aK4%uFfOWoA}4FQ zDXIW{M~vgpsA9Ik?Uui> z4Y{x4af9N46Ui>A8iU*I){+@RtXM^8G}1i+?*8uI_|V%;l6X-|3A({cW#^X$u#5@? zgm{dynD0?CjwUizNtupX%A@|iB_p7Pp=}?ER_iIJ9LTKk= zIFg>eM72KbOsNuCMyBc)OYBmPIm1>0LW z7ik5h-ZaeH+};ycME(4C5Dekj!+l%!ec8ZYJ%~U8d!AjzQ3{QN)JK2X`#Eo<5tFN7 z{`3<_(YV?-4J6g>thfZvhf|uxl1by+9rs&t_{lIs7j~c)l91&9w{^^5f{E;E@mYOD z{%J^4C!S=*U?jOvkM?s)3Fkmlv6jG6dLBcWzg|wetTq&e!)Y|;fWEed!0CCb71>{} zH_jFBIS!xUh3ahg)%p));3&IFF<<`jW$DXrmOc7uMsh$8NKnDzUvB%6EGzg%>1&p+ z8M6LF#mU?whx&@ZeOL$DCN8h#MeX)u^QR4}5lZluAGuo-T3p_bzE2g^92Ni^9>#RO z<&9`^s(iy0wFmo%<54t|$Rqa)vE$!gRl;$|=a^40wGiA!Lg1Sz`Dj zO0?9q5LD`!2D$H0=l8X`Ko2_EZj~)ixP5yx#3A&($eTbtavSge{@>!Z48e{wrfl}X zGNMHCL7$DsDdu-0!oPQSgeJwGcYmW^(DFi zY4)ADtNV6tG1|KLc68`Ags5hf%e!eBO^B{q{(t!y+g3HQ_Em<}y8%^(D=556oM=yw z*y}J6r}8vP%{qOUkfL3o6xSQZnW%!oP~v>M@;Q2~fWB1?>xD(-r$H@mTbZJK&C6!1 z^qCqu6;`*AfF_Hxu_-r}1dCoSx*>#F4@F>sDUv+k;cwXNlH(R1h3iS$0 z96ZIQkg;|HDbg-UC9m1hJ4`%m`#8{@ds=LOQXUNaO}XOf2yq9Gx$5W&_iM`&Z-wxY zk#LeNF>TV_17|O9K$1Ir?z{T^ifwDsJMYGqDY{H7NiT4qtf!x>xrbgLClHIsb4v|3 zw+ronxJNcc70($o{ZnqoaWAoTX@;&6ToueQ1Idtx1esgj5eaOG_p9x8a#^Ycazu%g z>eg+rA@(kIVvb3rLUhHubwC1DkZI1;DYpW6A;m|FO(f^|l>L58HbM#1WV+Tx{aK@K zaNY=otV^n78;6eb(-0RfLUHs7_BT{Rr!x}ZEia_Qn9l8TPMlZ)=S#8N3m{qn4O`j1uClXfyS>gtZ*j(6! zR80snHgLanE;+xCGhwai&S zx2+pCmGClIlEjf(2B68l3eH<(+4D4H6?g1GF=C0d448x7I7^ML+p0f;y6l+ zi}55Pq44Xh`;D2*6h|po;L?lgO>+Z)Bpt9!+7VRwngXKjWO z@FwNbUdsZNFq~i#o}w8}6H5fR5*z@QU+-69#nadGxZi7(+g=fv8~4qy5Z5pCzI>i> z-|JfX80Kv8tn&MBcK!bPM*hgMKd0H9keAVvr_@fiZiA|BOTQ_eMIe<5CCdI669lczV)V)4DjRF;ng1y`4(%N)SV_BG;uO#AlRSo&|ND$) z$?Nc0CPJ{dj9+iRjHh^hmiXK4KR8N-Sx_>QmoBtS654M-)ypGH7&!w@4=Fn!JmL%g zOXXhPVZ^vD!n|GnVmL=hwe2?`tsT(ddL@wW;%*j3m;~7Wq1nmicHn!QcfU z6&|S{bS+6R=6#j~x@HyRm4tLqbFA$9FxS1hmzLf_1%|T}pD|S)Rc3?k=?O^3UaeWe zt^cmobqJ$05yuX0#6{>V$ypl9GSU?DgM0|V^Sx4$FuWOv>6D07ho1y74t(Yk| zN?xp2hm6j|W7F~Dv&7>Ru)7-W)P{sugqZ< z*EdqfXtADVApUnL7HM(YpIB-ryZ2SqGYaW0$q*V`pgHu_HDY0GlyIb@!W~U)9IuhM zJU{5AnDv#(?|&|#fzZHAf+~tMdID9jT%{6*9C5s*$sqZZhK-G5YfNWJ`R70GP0R-W zznn>V;!aou;Rv2freA;s$WJmWZ4vBliD8`L`JRj}xQ3^f!?DBc>LVQR)Va-8dx+4L z0XJ3qU9qNLtgnBhDj_s(PFmAW_~i;-<=nxu*(vURZ8Aqr%D zYk>xI?sQsQKZ55Ht(UI$LL01L8y5HG!rxpdfS@ij9>pCM(oM>o;<*Hij`X26)NGYr z1sK|jG2Y5C3V@F!G0S&W{$BFJ$^eZczM>&Pjp7(W9I1UDx~k0ohqmkKahr(Z9H;~m zDu@cv14yN)QZ~}s@pNGqE_Ai+0%Du33erkk+=(;xn%W*SwiE51%8esuj(h_?4xfS} zS2SZ5G?3p*eU&7{?6GML4M+~X(uVC7I7G@z&h_?@0FOfD& z&(-v%sb2N`)CS)8p{EhNsmM2E=LSMF4Yj?1{|muh7D*%fe@)mFMPdzx->R`p*4>g7 zK@8o1nISKqyvrM~mn_V`3lV}+8d7}SJt^eP_GQR>c`edgV-63(@16g9Ii4XhXi$)3_W^j z;GlK5rN}E3frw5>R=wz}zTPR?3h5Lvv2rD7dL}(p>@r{XHCIbD}f?aq|b9+-qAHvpM->#fbpwq-|I3#%%s#C7#IuKvcIIj-tVu9^^r2Q#+a?(EfG z=#Hyo8{<7UOr&1DRg=B7O5X9RCK*19tZ5RY-8kfZJ)lha_eq#Lb10J8bbkF<(go=B*0HHnQ1-OK>x>F+Tg43yN%TtzleQ@4291;ErGF8$r~;QHCEB-HAhI@P|s`ih($ zJD@o~7|F0SV5BEsSVtHM>I2$B>Kxu`&K}#qJ*fVt6BQ{&Epo1oM3WMyOf+fWsaK3; z56c~5enfK+&j`J|BpiU4`plSZy$)qdAXmxC!mZQ&mZ>qDqX@&3%?yNz0%|lPxDC@5 zF)FgPm{MmqpzNSXBx1nHr>ktJErB05zph$m-p3#|ezUq;RQXJp3leefHNNP6;B~ z%zAy2$?7*Wj_iQTmq&!nh&H{FxwBnacIqQ*gc#P0Xo5@_GEjOQxBb4LnVz;dF%0G? z8psot%E|#JA1-2&kFZRJ@GvTi?>39caX+J2JK!bI83Jh^o)=lJ%RXxi6SBvHqFbEW zUYx3d_}u9BqgM6FY-D|DjU+;SBBd!$y`Xn5OOot5XJKLm478X#XivMKl(302szcf- z6*PvG7n+(<0|F&3S?` z(?~|#@YFQUe5Z^!3KrrE6Z*Q_T+_S-h;LlKxK%lhhwFKB?0|9YhzZ}%az|=lb{6q8 zy-_LBRlQi=QYME1*AqX)8Vr5GZ&{*d-VQ9dUGI=K?E=$sJS8|toAx$;={Lyam(;?e z9(ErEm;Upew8YgrKkmS0r0@Rt^WWe9`{%FU?*Dw}oxksW@b;@WZyuy?{T~1T|Nq~K JzKk!28vs<@$4>wN literal 0 HcmV?d00001 From 867d60c09abb5402951ae2700b87bdd1cae9f464 Mon Sep 17 00:00:00 2001 From: Gao Hongtao Date: Sun, 10 Nov 2024 22:30:52 +0800 Subject: [PATCH 2/6] Improve the index Signed-off-by: Gao Hongtao --- .golangci.yml | 195 ++++++++++----------- banyand/internal/storage/index.go | 2 +- go.mod | 10 +- go.sum | 64 ++----- pkg/index/index.go | 6 +- pkg/index/inverted/inverted.go | 46 ++--- pkg/index/inverted/inverted_series.go | 83 ++++++--- pkg/index/inverted/inverted_series_test.go | 2 +- scripts/build/vendor.mk | 2 +- 9 files changed, 185 insertions(+), 225 deletions(-) diff --git a/.golangci.yml b/.golangci.yml index 5c6e14338..009c83385 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -16,100 +16,100 @@ linters: disable-all: true enable: - - asasalint - - asciicheck - # - containedctx - - contextcheck - - decorder - - dogsled - - errname - - errcheck - - errchkjson - - errorlint - - exhaustive - # TODO:// enable this lint - # - exhaustruct - - gci - - goconst - - gocritic - - gocyclo - - godot - - gofumpt - - goprintffuncname - - gosec - - gosimple - - govet - - importas - - ineffassign - - lll - - misspell - - staticcheck - - stylecheck - - typecheck - - unconvert - - unparam - - unused - - usestdlibvars - - whitespace + - asasalint + - asciicheck + # - containedctx + - contextcheck + - decorder + - dogsled + - errname + - errcheck + - errchkjson + - errorlint + - exhaustive + # TODO:// enable this lint + # - exhaustruct + - gci + - goconst + - gocritic + - gocyclo + - godot + - gofumpt + - goprintffuncname + - gosec + - gosimple + - govet + - importas + - ineffassign + - lll + - misspell + - staticcheck + - stylecheck + - typecheck + - unconvert + - unparam + - unused + - usestdlibvars + - whitespace linters-settings: importas: no-unaliased: true alias: - - pkg: github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1 - alias: commonv1 - - pkg: github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1 - alias: databasev1 - - pkg: github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v1 - alias: modelv1 - - pkg: github.com/apache/skywalking-banyandb/api/proto/banyandb/property/v1 - alias: propertyv1 - - pkg: github.com/apache/skywalking-banyandb/api/proto/banyandb/measure/v1 - alias: measurev1 - - pkg: github.com/apache/skywalking-banyandb/api/proto/banyandb/stream/v1 - alias: streamv1 - - pkg: github.com/apache/skywalking-banyandb/api/proto/banyandb/cluster/v1 - alias: clusterv1 - - pkg: github.com/apache/skywalking-banyandb/pkg/pb/v1 - alias: pbv1 + - pkg: github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1 + alias: commonv1 + - pkg: github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1 + alias: databasev1 + - pkg: github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v1 + alias: modelv1 + - pkg: github.com/apache/skywalking-banyandb/api/proto/banyandb/property/v1 + alias: propertyv1 + - pkg: github.com/apache/skywalking-banyandb/api/proto/banyandb/measure/v1 + alias: measurev1 + - pkg: github.com/apache/skywalking-banyandb/api/proto/banyandb/stream/v1 + alias: streamv1 + - pkg: github.com/apache/skywalking-banyandb/api/proto/banyandb/cluster/v1 + alias: clusterv1 + - pkg: github.com/apache/skywalking-banyandb/pkg/pb/v1 + alias: pbv1 lll: line-length: 170 goconst: min-occurrences: 4 govet: enable: - - fieldalignment - - shadow + - fieldalignment + - shadow misspell: locale: US unparam: check-exported: false gci: sections: - - standard - - default - - prefix(github.com/apache/skywalking-banyandb/) + - standard + - default + - prefix(github.com/apache/skywalking-banyandb/) gocritic: enabled-checks: - - appendCombine - - boolExprSimplify - - builtinShadow - - commentedOutCode - - commentedOutImport - - docStub - - emptyFallthrough - - equalFold - - hexLiteral - - indexAlloc - - initClause - - methodExprCall - - nilValReturn - - octalLiteral - - rangeExprCopy - - stringXbytes - - typeAssertChain - - typeUnparen - - unnecessaryBlock - - weakCond + - appendCombine + - boolExprSimplify + - builtinShadow + - commentedOutCode + - commentedOutImport + - docStub + - emptyFallthrough + - equalFold + - hexLiteral + - indexAlloc + - initClause + - methodExprCall + - nilValReturn + - octalLiteral + - rangeExprCopy + - stringXbytes + - typeAssertChain + - typeUnparen + - unnecessaryBlock + - weakCond godot: # declarations - for top level declaration comments (default); # toplevel - for top level comments; @@ -117,38 +117,37 @@ linters-settings: scope: toplevel gosec: excludes: - - G115 # integer overflow conversion (TODO: verify these) + - G115 # integer overflow conversion (TODO: verify these) staticcheck: - checks: ["all", "-ST1000", "-ST1016", "-ST1020", "-ST1021", "-ST1022"] + checks: [ "all", "-ST1000", "-ST1016", "-ST1020", "-ST1021", "-ST1022" ] exhaustive: check: - - switch - - map + - switch + - map default-signifies-exhaustive: true ignore-enum-members: ".+UNSPECIFIED$" exhaustruct: exclude: - - 'cobra\.Command$' + - 'cobra\.Command$' run: go: "1.23" issues: exclude-rules: - - path: ".*\\.pb\\.go" - linters: - - "*" - - path: ".*\\.pb\\.validate\\.go" - linters: - - "*" - - path: ".*\\.gen\\.go" - linters: - - "*" - - path: _test\.go$|^tests/|^samples/ - linters: - - errcheck - - maligned - - linters: - - staticcheck - text: "SA1019: package github.com/golang/protobuf" - max-per-linter: 0 + - path: ".*\\.pb\\.go" + linters: + - "*" + - path: ".*\\.pb\\.validate\\.go" + linters: + - "*" + - path: ".*\\.gen\\.go" + linters: + - "*" + - path: _test\.go$|^tests/|^samples/ + linters: + - errcheck + - maligned + - linters: + - staticcheck + text: "SA1019: package github.com/golang/protobuf" max-same-issues: 0 diff --git a/banyand/internal/storage/index.go b/banyand/internal/storage/index.go index 2e1ad14b8..ef9639838 100644 --- a/banyand/internal/storage/index.go +++ b/banyand/internal/storage/index.go @@ -71,7 +71,7 @@ func newSeriesIndex(ctx context.Context, root string, flushTimeoutSeconds int64, } func (s *seriesIndex) Write(docs index.Documents) error { - return s.store.Batch(index.Batch{ + return s.store.SeriesBatch(index.Batch{ Documents: docs, }) } diff --git a/go.mod b/go.mod index c5c5a97fb..e59aa2962 100644 --- a/go.mod +++ b/go.mod @@ -96,7 +96,7 @@ require ( github.com/inconshreveable/mousetrap v1.1.0 // indirect github.com/jonboulle/clockwork v0.4.0 // indirect github.com/json-iterator/go v1.1.12 // indirect - github.com/klauspost/compress v1.17.9 + github.com/klauspost/compress v1.17.11 github.com/lufia/plan9stats v0.0.0-20240909124753-873cd0166683 // indirect github.com/magiconair/properties v1.8.7 // indirect github.com/mattn/go-colorable v0.1.13 // indirect @@ -144,8 +144,8 @@ require ( go.uber.org/zap v1.27.0 golang.org/x/crypto v0.27.0 // indirect golang.org/x/net v0.29.0 // indirect - golang.org/x/sys v0.25.0 - golang.org/x/text v0.18.0 // indirect + golang.org/x/sys v0.26.0 + golang.org/x/text v0.19.0 // indirect golang.org/x/time v0.6.0 // indirect golang.org/x/tools v0.25.0 // indirect google.golang.org/genproto v0.0.0-20240903143218-8af14fe29dc1 // indirect @@ -157,7 +157,7 @@ require ( replace ( github.com/benbjohnson/clock v1.3.0 => github.com/SkyAPM/clock v1.3.1-0.20220809233656-dc7607c94a97 - github.com/blugelabs/bluge => github.com/zinclabs/bluge v1.1.5 + github.com/blugelabs/bluge => github.com/SkyAPM/bluge v0.0.0-20241110125856-046bc03b30ab github.com/blugelabs/bluge_segment_api => github.com/zinclabs/bluge_segment_api v1.0.0 - github.com/blugelabs/ice => github.com/zinclabs/ice v1.1.3 + github.com/blugelabs/ice => github.com/SkyAPM/ice v0.0.0-20241108011032-c3d8eea75118 ) diff --git a/go.sum b/go.sum index 7872c57a0..a8a567a49 100644 --- a/go.sum +++ b/go.sum @@ -6,14 +6,15 @@ github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAE github.com/RoaringBitmap/roaring v0.9.4/go.mod h1:icnadbWcNyfEHlYdr+tDlOTih1Bf/h+rzPpv4sbomAA= github.com/RoaringBitmap/roaring v1.9.4 h1:yhEIoH4YezLYT04s1nHehNO64EKFTop/wBhxv2QzDdQ= github.com/RoaringBitmap/roaring v1.9.4/go.mod h1:6AXUsoIEzDTFFQCe1RbGA6uFONMhvejWj5rqITANK90= +github.com/SkyAPM/bluge v0.0.0-20241110125856-046bc03b30ab h1:iCQVR0hi6Kd4Pzh/QDHrBIK8TDTKzwEEzrYhdvqoPRg= +github.com/SkyAPM/bluge v0.0.0-20241110125856-046bc03b30ab/go.mod h1:6o9wC3xO3qb5Q7VmD1x0r54qQBDpO9+ghGAQvuOHsCU= github.com/SkyAPM/clock v1.3.1-0.20220809233656-dc7607c94a97 h1:FKuhJ+6n/DHspGeLleeNbziWnKr9gHKYN4q7NcoCp4s= github.com/SkyAPM/clock v1.3.1-0.20220809233656-dc7607c94a97/go.mod h1:2xGRl9H1pllhxTbEGO1W3gDkip8P9GQaHPni/wpdR44= -github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af/go.mod h1:K08gAheRH3/J6wwsYMMT4xOr94bZjxIelGM0+d/wbFw= +github.com/SkyAPM/ice v0.0.0-20241108011032-c3d8eea75118 h1:Ja62sgOCp2qPTd8Xmldv1U83v11IRIsh6KlB7UaFLj4= +github.com/SkyAPM/ice v0.0.0-20241108011032-c3d8eea75118/go.mod h1:DoQeb0Ee86LyruZSL77Ddscfk/THJ38x453CRCnGEPI= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/apache/skywalking-cli v0.0.0-20240227151024-ee371a210afe h1:zIc2yfpc/vMpfTtWprCVpca6CMJwb6X9cknqAoFeEFo= github.com/apache/skywalking-cli v0.0.0-20240227151024-ee371a210afe/go.mod h1:pu6Q19Xs38FSfy/IwnJGAMilO+W58/ugM8aMfLzw+i0= -github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= -github.com/axiomhq/hyperloglog v0.0.0-20191112132149-a4c4c47bc57f/go.mod h1:2stgcRjl6QmW+gU2h5E7BQXg4HU0gzxKWDuT5HviN9s= github.com/axiomhq/hyperloglog v0.2.0 h1:u1XT3yyY1rjzlWuP6NQIrV4bRYHOaqZaovqjcBEvZJo= github.com/axiomhq/hyperloglog v0.2.0/go.mod h1:GcgMjz9gaDKZ3G0UMS6Fq/VkZ4l7uGgcJyxA7M+omIM= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= @@ -25,15 +26,12 @@ github.com/bits-and-blooms/bitset v1.14.3 h1:Gd2c8lSNf9pKXom5JtD7AaKO8o7fGQ2LtFj github.com/bits-and-blooms/bitset v1.14.3/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= github.com/blevesearch/go-porterstemmer v1.0.3 h1:GtmsqID0aZdCSNiY8SkuPJ12pD4jI+DdXTAn4YRcHCo= github.com/blevesearch/go-porterstemmer v1.0.3/go.mod h1:angGc5Ht+k2xhJdZi511LtmxuEf0OVpvUUNrwmM1P7M= -github.com/blevesearch/mmap-go v1.0.3/go.mod h1:pYvKl/grLQrBxuaRYgoTssa4rVujYYeenDp++2E+yvs= github.com/blevesearch/mmap-go v1.0.4 h1:OVhDhT5B/M1HNPpYPBKIEJaD0F3Si+CrEKULGCDPWmc= github.com/blevesearch/mmap-go v1.0.4/go.mod h1:EWmEAOmdAS9z/pi/+Toxu99DnsbhG1TIxUoRmJw/pSs= -github.com/blevesearch/segment v0.9.0/go.mod h1:9PfHYUdQCgHktBgvtUOF4x+pc4/l8rdH0u5spnW85UQ= github.com/blevesearch/segment v0.9.1 h1:+dThDy+Lvgj5JMxhmOVlgFfkUtZV2kw49xax4+jTfSU= github.com/blevesearch/segment v0.9.1/go.mod h1:zN21iLm7+GnBHWTao9I+Au/7MBiL8pPFtJBJTsk6kQw= github.com/blevesearch/snowballstem v0.9.0 h1:lMQ189YspGP6sXvZQ4WZ+MLawfV8wOmPoD/iWeNXm8s= github.com/blevesearch/snowballstem v0.9.0/go.mod h1:PivSj3JMc8WuaFkTSRDW2SlrulNWPl4ABg1tC/hlgLs= -github.com/blevesearch/vellum v1.0.7/go.mod h1:doBZpmRhwTsASB4QdUZANlJvqVAUdUyX0ZK7QJCTeBE= github.com/blevesearch/vellum v1.0.10 h1:HGPJDT2bTva12hrHepVT3rOyIKFFF4t7Gf6yMxyMIPI= github.com/blevesearch/vellum v1.0.10/go.mod h1:ul1oT0FhSMDIExNjIxHqJoGpVrBpKCdgDQNxfqgJt7k= github.com/caio/go-tdigest v3.1.0+incompatible h1:uoVMJ3Q5lXmVLCCqaMGHLBWnbGoN6Lpu7OAUPR60cds= @@ -51,14 +49,10 @@ github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnht github.com/cncf/xds/go v0.0.0-20210312221358-fbca930ec8ed/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cockroachdb/datadriven v1.0.2 h1:H9MtNqVoVhvd9nCBwOyDjUEdZCREqbIdCJD93PBm/jA= github.com/cockroachdb/datadriven v1.0.2/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU= -github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= -github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= -github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-semver v0.3.1 h1:yi21YpKnrx1gt5R+la8n5WgS0kCrsPp33dmEyHReZr4= github.com/coreos/go-semver v0.3.1/go.mod h1:irMmmIw/7yzSRPWryHsK7EYSg09caPQL03VsM8rvUec= github.com/coreos/go-systemd/v22 v22.5.0 h1:RrqgGjYQKalulkV8NGVIfkXQf6YYmOyiJKk8iXXhfZs= github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= -github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= github.com/cpuguy83/go-md2man/v2 v2.0.4 h1:wfIWP927BUkWJb2NmU/kNDYIBTh/ziUX91+lVfRxZq4= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -68,7 +62,6 @@ github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8Yc github.com/dchest/siphash v1.2.2/go.mod h1:q+IRvb2gOSrUnYoPqHiyHXS0FOBBOdl6tONBlVnOnt4= github.com/dchest/siphash v1.2.3 h1:QXwFc8cFOR2dSa/gE6o/HokBMWtLUaNDVd+22aKHeEA= github.com/dchest/siphash v1.2.3/go.mod h1:0NvQU092bT0ipiFN++/rXm69QG9tVxLAlQHIXMPAkHc= -github.com/dgryski/go-metro v0.0.0-20180109044635-280f6062b5bc/go.mod h1:c9O8+fpSOX1DM8cPNSkX/qsBWdkD4yd2dpciOWQjpBw= github.com/dgryski/go-metro v0.0.0-20211217172704-adc40b04c140 h1:y7y0Oa6UawqTFPCDw9JG6pdKt4F9pAhHv0B7FMGaGD0= github.com/dgryski/go-metro v0.0.0-20211217172704-adc40b04c140/go.mod h1:c9O8+fpSOX1DM8cPNSkX/qsBWdkD4yd2dpciOWQjpBw= github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= @@ -84,10 +77,8 @@ github.com/envoyproxy/go-control-plane v0.9.9-0.20210512163311-63b5d3c536b0/go.m github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/envoyproxy/protoc-gen-validate v1.1.0 h1:tntQDh69XqOCOZsDz0lVJQez/2L6Uu2PdjCQwWCJ3bM= github.com/envoyproxy/protoc-gen-validate v1.1.0/go.mod h1:sXRDRVmzEbkM7CVcM06s9shE/m23dg3wzjl0UWqJ2q4= -github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/frankban/quicktest v1.14.6 h1:7Xjx+VpznH+oBnejlPUj8oUpdxnVs4f8XU8WnHkI4W8= github.com/frankban/quicktest v1.14.6/go.mod h1:4ptaffx2x8+WTWXmUCuVU6aPUX1/Mz7zb5vbUoiM6w0= -github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.7.0 h1:8JEhPFa5W2WU7YfeZzPNqzMP6Lwt7L2715Ggo0nosvA= github.com/fsnotify/fsnotify v1.7.0/go.mod h1:40Bi/Hjc2AVfZrqy+aj+yEI+/bRxZnMJyTJwOpGvigM= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= @@ -113,7 +104,6 @@ github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang-jwt/jwt/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOWzg= github.com/golang-jwt/jwt/v4 v4.5.0/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= -github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -168,22 +158,18 @@ github.com/hashicorp/golang-lru v1.0.2 h1:dV3g9Z/unq5DpblPpw+Oqcv4dU/1omnb4Ok8iP github.com/hashicorp/golang-lru v1.0.2/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= github.com/hashicorp/hcl v1.0.0 h1:0Anlzjpi4vEasTeNFn2mLJgTSwt0+6sfsiTG8qcWGx4= github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= -github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/inconshreveable/mousetrap v1.1.0 h1:wN+x4NVGpMsO7ErUn/mUI3vEoE6Jt13X2s0bqwp9tc8= github.com/inconshreveable/mousetrap v1.1.0/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw= -github.com/influxdata/influxdb v1.7.6/go.mod h1:qZna6X/4elxqT3yI9iZYdZrWWdeFOOprn86kgg4+IzY= github.com/jonboulle/clockwork v0.4.0 h1:p4Cf1aMWXnXAUh8lVfewRBx1zaTSYKrKMF2g3ST4RZ4= github.com/jonboulle/clockwork v0.4.0/go.mod h1:xgRqUGwRcjKCO1vbZUEtSLrqKoPSsUpK7fnezOII0kc= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= -github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/kkdai/maglev v0.2.0 h1:w6DCW0kAA6fstZqXkrBrlgIC3jeIRXkjOYea/m6EK/Y= github.com/kkdai/maglev v0.2.0/go.mod h1:d+mt8Lmt3uqi9aRb/BnPjzD0fy+ETs1vVXiGRnqHVZ4= -github.com/klauspost/compress v1.15.2/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= -github.com/klauspost/compress v1.17.9 h1:6KIumPrER1LHsvBVuDa0r5xaG0Es51mhhB9BQB2qeMA= -github.com/klauspost/compress v1.17.9/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= +github.com/klauspost/compress v1.17.11 h1:In6xLpyWOi1+C7tXUUWv2ot1QvBjxevKAaI6IXrJmUc= +github.com/klauspost/compress v1.17.11/go.mod h1:pMDklpSncoRMuLFrf1W9Ss9KT+0rH90U12bZKk7uwG0= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= @@ -200,7 +186,6 @@ github.com/lufia/plan9stats v0.0.0-20240909124753-873cd0166683 h1:7UMa6KCCMjZEMD github.com/lufia/plan9stats v0.0.0-20240909124753-873cd0166683/go.mod h1:ilwx/Dta8jXAgpFYFvSWEMwxmbWXyiUHkd5FwyKhb5k= github.com/machinebox/graphql v0.2.2 h1:dWKpJligYKhYKO5A2gvNhkJdQMNZeChZYyBbrZkBZfo= github.com/machinebox/graphql v0.2.2/go.mod h1:F+kbVMHuwrQ5tYgU9JXlnskM8nOaFxCAEolaQybkjWA= -github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/magiconair/properties v1.8.7 h1:IeQXZAiQcpL9mgcAe1Nu6cX9LLw6ExEHKjN0VQdvPDY= github.com/magiconair/properties v1.8.7/go.mod h1:Dhd985XPs7jluiymwWYZ0G4Z61jb3vdS329zhj2hYo0= github.com/matryer/is v1.4.0 h1:sosSmIWwkYITGrxZ25ULNDeKiMNzFSr4V/eqBQP0PeE= @@ -211,8 +196,6 @@ github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/ github.com/mattn/go-isatty v0.0.19/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= github.com/mattn/go-isatty v0.0.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY= github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= -github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= -github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -233,7 +216,6 @@ github.com/onsi/ginkgo/v2 v2.20.2/go.mod h1:K9gyxPIlb+aIvnZ8bd9Ak+YP18w3APlR+5co github.com/onsi/gomega v1.34.2 h1:pNCwDkzrsv7MS9kpaQvVb1aVLahQXyJ/Tv5oAZMI3i8= github.com/onsi/gomega v1.34.2/go.mod h1:v1xfxRgk0KIsG+QOdm7p8UosrOzPYRo60fd3B/1Dukc= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/pelletier/go-toml/v2 v2.2.3 h1:YmeHyLY8mFWbdkNWwpr+qIL2bEqT0o95WSdkNHvL12M= github.com/pelletier/go-toml/v2 v2.2.3/go.mod h1:MfCQTFTvCcUyyvvwm1+G6H/jORL20Xlb6rzQu9GuUkc= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -261,7 +243,6 @@ github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99 github.com/rs/xid v1.5.0/go.mod h1:trrq9SKmegXys3aeAKXMUTdJsYXVwGY3RLcfgqegfbg= github.com/rs/zerolog v1.33.0 h1:1cU2KZkvPxNyfgEmhHAz/1A9Bz+llsdYzklWFzgp0r8= github.com/rs/zerolog v1.33.0/go.mod h1:/7mN4D5sKwJLZQ2b/znpjC3/GQWY/xaDXUM0kKWRHss= -github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= github.com/russross/blackfriday/v2 v2.1.0 h1:JIOH55/0cWyOuilr9/qlrm0BSXldqnqwMsf35Ld67mk= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/sagikazarmark/locafero v0.6.0 h1:ON7AQg37yzcRPU69mt7gwhFEBwxI6P9T4Qu3N51bwOk= @@ -285,20 +266,14 @@ github.com/sourcegraph/conc v0.3.0/go.mod h1:Sdozi7LEKbFPqYX2/J+iBAM6HpqSLTASQIK github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= -github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= github.com/spf13/afero v1.11.0 h1:WJQKhtpdm3v2IzqG8VMqrr6Rf3UYpEF239Jy9wNepM8= github.com/spf13/afero v1.11.0/go.mod h1:GH9Y3pIexgf1MTIWtNGyogA5MwRIDXGUr+hbWNoBjkY= -github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cast v1.7.0 h1:ntdiHjuueXFgm5nzDRdOS4yfT43P5Fnud6DH50rz/7w= github.com/spf13/cast v1.7.0/go.mod h1:ancEpBxwJDODSW/UG4rDrAqiKolqNNh2DX3mk86cAdo= -github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= github.com/spf13/cobra v1.8.1 h1:e5/vxKd/rZsfSJMUX1agtjeTDf+qv1/JdBF8gg5k9ZM= github.com/spf13/cobra v1.8.1/go.mod h1:wHxEcudfqmLYa8iTfL+OuZPbBZkmvliBWKIezN3kD9Y= -github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= -github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= -github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= github.com/spf13/viper v1.19.0 h1:RWq5SEjt8o25SROyN3z2OrDB9l7RPd3lwTWU8EcEdcI= github.com/spf13/viper v1.19.0/go.mod h1:GQUN9bilAbhU/jgc1bKs99f/suXKeUMct8Adx5+Ntkg= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= @@ -318,14 +293,12 @@ github.com/tklauser/numcpus v0.8.0 h1:Mx4Wwe/FjZLeQsK/6kt2EOepwwSl7SmJrK5bV/dXYg github.com/tklauser/numcpus v0.8.0/go.mod h1:ZJZlAY+dmR4eut8epnzf0u/VwodKmryxR8txiloSqBE= github.com/tmc/grpc-websocket-proxy v0.0.0-20220101234140-673ab2c3ae75 h1:6fotK7otjonDflCTK0BCfls4SPy3NcCVb5dqqmbRknE= github.com/tmc/grpc-websocket-proxy v0.0.0-20220101234140-673ab2c3ae75/go.mod h1:KO6IkyS8Y3j8OdNO85qEYBsRPuteD+YciPomcXdrMnk= -github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/urfave/cli/v2 v2.27.4 h1:o1owoI+02Eb+K107p27wEX9Bb8eqIoZCfLXloLUSWJ8= github.com/urfave/cli/v2 v2.27.4/go.mod h1:m4QzxcD2qpra4z7WhzEGn74WZLViBnMpb1ToCAKdGRQ= github.com/xhit/go-str2duration/v2 v2.1.0 h1:lxklc02Drh6ynqX+DdPyp5pCKLUQpRT8bp8Ydu2Bstc= github.com/xhit/go-str2duration/v2 v2.1.0/go.mod h1:ohY8p+0f07DiV6Em5LKB0s2YpLtXVyJfNt1+BlmyAsU= github.com/xiang90/probing v0.0.0-20221125231312-a49e3df8f510 h1:S2dVYn90KE98chqDkyE9Z4N61UnQd+KOfgp5Iu53llk= github.com/xiang90/probing v0.0.0-20221125231312-a49e3df8f510/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= -github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= github.com/xrash/smetrics v0.0.0-20240521201337-686a1a2994c1 h1:gEOO8jv9F4OT7lGCjxCBTO/36wtF6j2nSip77qHd4x4= github.com/xrash/smetrics v0.0.0-20240521201337-686a1a2994c1/go.mod h1:Ohn+xnUBiLI6FVj/9LpzZWtj1/D6lUovWYBkxHVV3aM= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= @@ -336,12 +309,8 @@ github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= github.com/zenizh/go-capturer v0.0.0-20211219060012-52ea6c8fed04 h1:qXafrlZL1WsJW5OokjraLLRURHiw0OzKHD/RNdspp4w= github.com/zenizh/go-capturer v0.0.0-20211219060012-52ea6c8fed04/go.mod h1:FiwNQxz6hGoNFBC4nIx+CxZhI3nne5RmIOlT/MXcSD4= -github.com/zinclabs/bluge v1.1.5 h1:QJhkweeBVRaaEPdaRptkYOJDLCeyo+JBgc2hNyFehAM= -github.com/zinclabs/bluge v1.1.5/go.mod h1:IG9JlDUzUGRIxylWmLyF7e1QwMdRWu9FkNSivJ4VB+E= github.com/zinclabs/bluge_segment_api v1.0.0 h1:GJvPxdzR7KjwdxmcKleQLvtIYi/J7Q7ehRlZqgGayzg= github.com/zinclabs/bluge_segment_api v1.0.0/go.mod h1:mYfPVUdXLZ4iXsicXMER+RcI/avwphjMOi8nhN9HDLA= -github.com/zinclabs/ice v1.1.3 h1:LNfncdxQw2ix6P1T2ISmhO+6BFRa27qyTTfK0PitF2c= -github.com/zinclabs/ice v1.1.3/go.mod h1:wTwGEe30mQnSLaR1ezxu4E80GcwO6EyOww67KpJtIiw= go.etcd.io/bbolt v1.3.11 h1:yGEzV1wPz2yVCLsD8ZAiGHhHVlczyC9d1rP43/VCRJ0= go.etcd.io/bbolt v1.3.11/go.mod h1:dksAq7YMXoljX0xu6VF5DMZGbhYYoLUalEiSySYAS4I= go.etcd.io/etcd/api/v3 v3.5.16 h1:WvmyJVbjWqK4R1E+B12RRHz3bRGy9XVfh++MgbN+6n0= @@ -387,7 +356,6 @@ go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN8 go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= -golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= @@ -399,13 +367,9 @@ golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v golang.org/x/crypto v0.25.0/go.mod h1:T+wALwcMOSE0kXgUAnPAHqTLW+XHgcELELW8VaDgm/M= golang.org/x/crypto v0.27.0 h1:GXm2NjJrPaiv/h1tb2UH8QfgC/hOf/+z0p6PT8o1w7A= golang.org/x/crypto v0.27.0/go.mod h1:1Xngt8kV6Dvbssa53Ziq6Eqn0HqbZi5Z6R0ZpwQzt70= -golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20190125153040-c74c464bbbf2/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20240909161429-701f63a606c0 h1:e66Fs6Z+fZTbFBAxKfP3PALWBtpfqks2bwGcexMxgtk= golang.org/x/exp v0.0.0-20240909161429-701f63a606c0/go.mod h1:2TbTHSBQa924w8M6Xs1QcRcFwyucIwBGpK1p2f1YFFY= -golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= @@ -463,11 +427,9 @@ golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -492,8 +454,8 @@ golang.org/x/sys v0.16.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.25.0 h1:r+8e+loiHxRqhXVl6ML1nO3l1+oFoWbnlu2Ehimmi34= -golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= +golang.org/x/sys v0.26.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/telemetry v0.0.0-20240228155512-f48c80bd79b2/go.mod h1:TeRTkGYfJXctD9OcfyVLyj2J3IxLnKwHJR8f4D8a3YE= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -515,14 +477,12 @@ golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.15.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= -golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= -golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= +golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.6.0 h1:eTDhh4ZXt5Qf0augr54TN6suAUudPcawVZeIAPU7D4U= golang.org/x/time v0.6.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= -golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190206041539-40960b6deb8e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= @@ -542,11 +502,8 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= gonum.org/v1/gonum v0.7.0 h1:Hdks0L0hgznZLG9nzXb8vZ0rRvqNvAcgAp84y7Mwkgw= gonum.org/v1/gonum v0.7.0/go.mod h1:L02bwd0sqlsvRv41G7wGWFCsVNZFv/k1xzGIxeANHGM= -gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= -gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= @@ -605,7 +562,6 @@ gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= sigs.k8s.io/yaml v1.4.0 h1:Mk1wCc2gy/F0THH0TAp1QYyJNzRm2KCLy3o5ASXVI5E= sigs.k8s.io/yaml v1.4.0/go.mod h1:Ejl7/uTz7PSA4eKMyQCUTnhZYNmLIl+5c2lQPGR2BPY= skywalking.apache.org/repo/goapi v0.0.0-20240604102541-64f9001abe03 h1:y0f+3gNmeyC/V5Bt8VE9aL9mmWESj+WvyqJ2tOb7qrk= diff --git a/pkg/index/index.go b/pkg/index/index.go index 035fbf83d..fdcb2c927 100644 --- a/pkg/index/index.go +++ b/pkg/index/index.go @@ -58,11 +58,6 @@ func (f FieldKey) Marshal() string { return string(convert.Uint32ToBytes(f.IndexRuleID)) } -// HasSeriesID reports whether f has a series id. -func (f FieldKey) HasSeriesID() bool { - return f.SeriesID > 0 -} - // Field is a indexed item in a document. type Field struct { Term []byte @@ -169,6 +164,7 @@ type Batch struct { // Writer allows writing fields and docID in a document to an index. type Writer interface { Batch(batch Batch) error + SeriesBatch(batch Batch) error } // FieldIterable allows building a FieldIterator. diff --git a/pkg/index/inverted/inverted.go b/pkg/index/inverted/inverted.go index 094f405be..084ef7290 100644 --- a/pkg/index/inverted/inverted.go +++ b/pkg/index/inverted/inverted.go @@ -113,12 +113,8 @@ func (s *store) Batch(batch index.Batch) error { b := generateBatch() defer releaseBatch(b) for _, d := range batch.Documents { - var fk *index.FieldKey - if len(d.Fields) > 0 { - fk = &d.Fields[0].Key - } doc := bluge.NewDocument(convert.BytesToString(convert.Uint64ToBytes(d.DocID))) - for _, f := range d.Fields { + for i, f := range d.Fields { tf := bluge.NewKeywordFieldBytes(f.Key.Marshal(), f.Term) if !f.NoSort { tf.Sortable() @@ -130,17 +126,15 @@ func (s *store) Batch(batch index.Batch) error { tf = tf.WithAnalyzer(Analyzers[f.Key.Analyzer]) } doc.AddField(tf) + if i == 0 { + doc.AddField(bluge.NewKeywordFieldBytes(seriesIDField, f.Key.SeriesID.Marshal()).StoreValue()) + } } - if d.EntityValues != nil { - doc.AddField(bluge.NewKeywordFieldBytes(entityField, d.EntityValues).StoreValue()) - } else if fk != nil && fk.HasSeriesID() { - doc.AddField(bluge.NewKeywordFieldBytes(seriesIDField, fk.SeriesID.Marshal()).StoreValue()) - } if d.Timestamp > 0 { doc.AddField(bluge.NewDateTimeField(timestampField, time.Unix(0, d.Timestamp)).StoreValue()) } - b.Update(doc.ID(), doc) + b.Insert(doc) } return s.writer.Batch(b) } @@ -196,14 +190,18 @@ func (s *store) Iterator(ctx context.Context, fieldKey index.FieldKey, termRange fk := fieldKey.Marshal() rangeQuery := bluge.NewBooleanQuery() rangeNode := newMustNode() - addRange := func(query *bluge.BooleanQuery, termRange index.RangeOpts) *bluge.BooleanQuery { + + rangeQuery = rangeQuery.AddMust(bluge.NewTermQuery(string(fieldKey.SeriesID.Marshal())). + SetField(seriesIDField)) + rangeNode.Append(newTermNode(string(fieldKey.SeriesID.Marshal()), nil)) + if termRange.Lower != nil || termRange.Upper != nil { if termRange.Upper == nil { termRange.Upper = defaultUpper } if termRange.Lower == nil { termRange.Lower = defaultLower } - query.AddMust(bluge.NewTermRangeInclusiveQuery( + rangeQuery.AddMust(bluge.NewTermRangeInclusiveQuery( string(termRange.Lower), string(termRange.Upper), termRange.IncludesLower, @@ -211,18 +209,6 @@ func (s *store) Iterator(ctx context.Context, fieldKey index.FieldKey, termRange ). SetField(fk)) rangeNode.Append(newTermRangeInclusiveNode(string(termRange.Lower), string(termRange.Upper), termRange.IncludesLower, termRange.IncludesUpper, nil)) - return query - } - - if fieldKey.HasSeriesID() { - rangeQuery = rangeQuery.AddMust(bluge.NewTermQuery(string(fieldKey.SeriesID.Marshal())). - SetField(seriesIDField)) - rangeNode.Append(newTermNode(string(fieldKey.SeriesID.Marshal()), nil)) - if termRange.Lower != nil || termRange.Upper != nil { - rangeQuery = addRange(rangeQuery, termRange) - } - } else { - rangeQuery = addRange(rangeQuery, termRange) } sortedKey := fk @@ -252,11 +238,9 @@ func (s *store) MatchTerms(field index.Field) (list posting.List, err error) { } fk := field.Key.Marshal() query := bluge.NewBooleanQuery(). - AddMust(bluge.NewTermQuery(string(field.Term)).SetField(fk)) - if field.Key.HasSeriesID() { - query = query.AddMust(bluge.NewTermQuery(string(field.Key.SeriesID.Marshal())). + AddMust(bluge.NewTermQuery(string(field.Term)).SetField(fk)). + AddMust(bluge.NewTermQuery(string(field.Key.SeriesID.Marshal())). SetField(seriesIDField)) - } documentMatchIterator, err := reader.Search(context.Background(), bluge.NewAllMatches(query)) if err != nil { return nil, err @@ -283,9 +267,7 @@ func (s *store) Match(fieldKey index.FieldKey, matches []string, opts *modelv1.C analyzer, operator := getMatchOptions(fieldKey.Analyzer, opts) fk := fieldKey.Marshal() query := bluge.NewBooleanQuery() - if fieldKey.HasSeriesID() { - query.AddMust(bluge.NewTermQuery(string(fieldKey.SeriesID.Marshal())).SetField(seriesIDField)) - } + query.AddMust(bluge.NewTermQuery(string(fieldKey.SeriesID.Marshal())).SetField(seriesIDField)) for _, m := range matches { query.AddMust(bluge.NewMatchQuery(m).SetField(fk). SetAnalyzer(analyzer).SetOperator(operator)) diff --git a/pkg/index/inverted/inverted_series.go b/pkg/index/inverted/inverted_series.go index fcf8f4672..10c7718c4 100644 --- a/pkg/index/inverted/inverted_series.go +++ b/pkg/index/inverted/inverted_series.go @@ -22,6 +22,7 @@ import ( "bytes" "context" "io" + "time" "github.com/blugelabs/bluge" "github.com/blugelabs/bluge/search" @@ -37,6 +38,41 @@ import ( var emptySeries = make([]index.SeriesDocument, 0) +func (s *store) SeriesBatch(batch index.Batch) error { + if len(batch.Documents) == 0 { + return nil + } + if !s.closer.AddRunning() { + return nil + } + defer s.closer.Done() + b := generateBatch() + defer releaseBatch(b) + for _, d := range batch.Documents { + doc := bluge.NewDocument(convert.BytesToString(convert.Uint64ToBytes(d.DocID))) + for _, f := range d.Fields { + tf := bluge.NewKeywordFieldBytes(f.Key.Marshal(), f.Term) + if !f.NoSort { + tf.Sortable() + } + if f.Store { + tf.StoreValue() + } + if f.Key.Analyzer != index.AnalyzerUnspecified { + tf = tf.WithAnalyzer(Analyzers[f.Key.Analyzer]) + } + doc.AddField(tf) + } + + doc.AddField(bluge.NewKeywordFieldBytes(entityField, d.EntityValues).StoreValue()) + if d.Timestamp > 0 { + doc.AddField(bluge.NewDateTimeField(timestampField, time.Unix(0, d.Timestamp)).StoreValue()) + } + b.InsertIfAbsent(doc.ID(), doc) + } + return s.writer.Batch(b) +} + // BuildQuery implements index.SeriesStore. func (s *store) BuildQuery(seriesMatchers []index.SeriesMatcher, secondaryQuery index.Query) (index.Query, error) { if len(seriesMatchers) == 0 { @@ -104,6 +140,10 @@ func (s *store) Search(ctx context.Context, return nil, err } defer func() { + if err := recover(); err != nil { + _ = reader.Close() + panic(err) + } _ = reader.Close() }() @@ -183,36 +223,23 @@ func (s *store) SeriesSort(ctx context.Context, fieldKey index.FieldKey, termRan return nil, err } fk := fieldKey.Marshal() - rangeQuery := bluge.NewBooleanQuery() - rangeNode := newMustNode() - addRange := func(query *bluge.BooleanQuery, termRange index.RangeOpts) *bluge.BooleanQuery { - if termRange.Upper == nil { - termRange.Upper = defaultUpper - } - if termRange.Lower == nil { - termRange.Lower = defaultLower - } - query.AddMust(bluge.NewTermRangeInclusiveQuery( - string(termRange.Lower), - string(termRange.Upper), - termRange.IncludesLower, - termRange.IncludesUpper, - ). - SetField(fk)) - rangeNode.Append(newTermRangeInclusiveNode(string(termRange.Lower), string(termRange.Upper), termRange.IncludesLower, termRange.IncludesUpper, nil)) - return query - } - if fieldKey.HasSeriesID() { - rangeQuery = rangeQuery.AddMust(bluge.NewTermQuery(string(fieldKey.SeriesID.Marshal())). - SetField(seriesIDField)) - rangeNode.Append(newTermNode(string(fieldKey.SeriesID.Marshal()), nil)) - if termRange.Lower != nil || termRange.Upper != nil { - rangeQuery = addRange(rangeQuery, termRange) - } - } else { - rangeQuery = addRange(rangeQuery, termRange) + if termRange.Upper == nil { + termRange.Upper = defaultUpper + } + if termRange.Lower == nil { + termRange.Lower = defaultLower } + rangeQuery := bluge.NewBooleanQuery() + rangeQuery.AddMust(bluge.NewTermRangeInclusiveQuery( + string(termRange.Lower), + string(termRange.Upper), + termRange.IncludesLower, + termRange.IncludesUpper, + ). + SetField(fk)) + rangeNode := newMustNode() + rangeNode.Append(newTermRangeInclusiveNode(string(termRange.Lower), string(termRange.Upper), termRange.IncludesLower, termRange.IncludesUpper, nil)) sortedKey := fk if order == modelv1.Sort_SORT_DESC { diff --git a/pkg/index/inverted/inverted_series_test.go b/pkg/index/inverted/inverted_series_test.go index 44fe96716..c0e09e1e5 100644 --- a/pkg/index/inverted/inverted_series_test.go +++ b/pkg/index/inverted/inverted_series_test.go @@ -393,7 +393,7 @@ func setupData(tester *require.Assertions, s index.SeriesStore) { }, }, } - tester.NoError(s.Batch(index.Batch{ + tester.NoError(s.SeriesBatch(index.Batch{ Documents: []index.Document{series1, series2, series3, series3}, })) } diff --git a/scripts/build/vendor.mk b/scripts/build/vendor.mk index 2823aaf86..a8773ac16 100644 --- a/scripts/build/vendor.mk +++ b/scripts/build/vendor.mk @@ -21,5 +21,5 @@ .PHONY: vendor-update vendor-update: ## Update vendors - go get -u -d ./... + go get -u ./... go mod tidy -compat=1.23 From e3f4e60e611972b989ad152c283d00da986eb699 Mon Sep 17 00:00:00 2001 From: Gao Hongtao Date: Wed, 13 Nov 2024 14:00:22 +0800 Subject: [PATCH 3/6] Change the options to "index_mode" Signed-off-by: Gao Hongtao --- api/proto/banyandb/database/v1/schema.proto | 5 +- banyand/measure/measure.go | 2 +- banyand/measure/query.go | 36 +++---- banyand/measure/write.go | 113 ++++++++++---------- banyand/stream/stream.go | 2 +- dist/LICENSE | 2 +- docs/api-reference.md | 2 +- go.mod | 2 +- go.sum | 6 +- pkg/index/index.go | 4 +- pkg/index/inverted/inverted.go | 5 +- pkg/index/inverted/inverted_series.go | 35 ++---- pkg/index/inverted/sort.go | 2 +- pkg/partition/index.go | 5 +- pkg/query/model/model.go | 20 ---- 15 files changed, 105 insertions(+), 136 deletions(-) diff --git a/api/proto/banyandb/database/v1/schema.proto b/api/proto/banyandb/database/v1/schema.proto index 83fbeb238..75876e3ab 100644 --- a/api/proto/banyandb/database/v1/schema.proto +++ b/api/proto/banyandb/database/v1/schema.proto @@ -115,8 +115,9 @@ message Measure { string interval = 5; // updated_at indicates when the measure is updated google.protobuf.Timestamp updated_at = 6; - // non_time_series indicates whether the measure is a time series - bool non_time_series = 7; + // index_mode specifies whether the data should be stored exclusively in the index, + // meaning it will not be stored in the data storage system. + bool index_mode = 7; } message MeasureAggregateFunction { diff --git a/banyand/measure/measure.go b/banyand/measure/measure.go index f278735e4..40a108982 100644 --- a/banyand/measure/measure.go +++ b/banyand/measure/measure.go @@ -103,7 +103,7 @@ func (s *measure) parseSpec() (err error) { if s.schema.Interval != "" { s.interval, err = timestamp.ParseDuration(s.schema.Interval) } - s.indexRuleLocators, s.fieldIndexLocation = partition.ParseIndexRuleLocators(s.schema.GetEntity(), s.schema.GetTagFamilies(), s.indexRules) + s.indexRuleLocators, s.fieldIndexLocation = partition.ParseIndexRuleLocators(s.schema.GetEntity(), s.schema.GetTagFamilies(), s.indexRules, s.schema.IndexMode) return err } diff --git a/banyand/measure/query.go b/banyand/measure/query.go index 457e90660..323993323 100644 --- a/banyand/measure/query.go +++ b/banyand/measure/query.go @@ -90,10 +90,10 @@ func (s *measure) Query(ctx context.Context, mqo model.MeasureQueryOptions) (mqr tsdb := db.(storage.TSDB[*tsTable, option]) segments := tsdb.SelectSegments(*mqo.TimeRange) if len(segments) < 1 { - return model.BypassResult, nil + return nil, nil } - if s.schema.NonTimeSeries { + if s.schema.IndexMode { return s.buildIndexQueryResult(ctx, series, mqo, segments) } @@ -102,12 +102,15 @@ func (s *measure) Query(ctx context.Context, mqo model.MeasureQueryOptions) (mqr return nil, err } if len(sids) < 1 { - return model.BypassResult, nil + for i := range segments { + segments[i].DecRef() + } + return nil, nil } result := queryResult{ ctx: ctx, segments: segments, - tagProjection: newTagProjection, + tagProjection: mqo.TagProjection, storedIndexValue: storedIndexValue, } defer func() { @@ -682,25 +685,20 @@ func (qr *queryResult) merge(storedIndexValue map[common.SeriesID]map[string]*mo return result } -var ( - bypassVersions = []int64{0} - bypassFields = []model.Field{} -) +var bypassVersions = []int64{0} type indexQueryResult struct { - mqo model.MeasureQueryOptions + ctx context.Context + err error tfl []tagFamilyLocation indexProjection []index.FieldKey - ctx context.Context series []*pbv1.Series - - segments []storage.Segment[*tsTable, option] - - i int - sll pbv1.SeriesList - frl storage.FieldResultList - timestamps []int64 - err error + segments []storage.Segment[*tsTable, option] + sll pbv1.SeriesList + frl storage.FieldResultList + timestamps []int64 + mqo model.MeasureQueryOptions + i int } // Pull implements model.MeasureQueryResult. @@ -768,7 +766,7 @@ func (i *indexQueryResult) Release() { } type tagFamilyLocation struct { - name string fieldToValueType map[string]tagNameWithType projectedEntityOffsets map[string]int + name string } diff --git a/banyand/measure/write.go b/banyand/measure/write.go index 6f5a96591..ab0879a22 100644 --- a/banyand/measure/write.go +++ b/banyand/measure/write.go @@ -125,6 +125,61 @@ func (w *writeCallback) handle(dst map[string]*dataPointsInGroup, writeEvent *me )) } dpt.dataPoints.fields = append(dpt.dataPoints.fields, field) + tagFamily, fields := w.handleTagFamily(stm, req) + dpt.dataPoints.tagFamilies = append(dpt.dataPoints.tagFamilies, tagFamily) + + if stm.processorManager != nil { + stm.processorManager.onMeasureWrite(uint64(series.ID), &measurev1.InternalWriteRequest{ + Request: &measurev1.WriteRequest{ + Metadata: stm.GetSchema().Metadata, + DataPoint: req.DataPoint, + MessageId: uint64(time.Now().UnixNano()), + }, + EntityValues: writeEvent.EntityValues, + }) + } + + doc := index.Document{ + DocID: uint64(series.ID), + EntityValues: series.Buffer, + Fields: fields, + } + if stm.schema.IndexMode { + doc.Timestamp = ts + } + dpg.docs = append(dpg.docs, doc) + + return dst, nil +} + +func (w *writeCallback) newDpt(tsdb storage.TSDB[*tsTable, option], dpg *dataPointsInGroup, t time.Time, ts int64, shardID common.ShardID) (*dataPointsInTable, error) { + var segment storage.Segment[*tsTable, option] + for _, seg := range dpg.segments { + if seg.GetTimeRange().Contains(ts) { + segment = seg + } + } + if segment == nil { + var err error + segment, err = tsdb.CreateSegmentIfNotExist(t) + if err != nil { + return nil, fmt.Errorf("cannot create segment: %w", err) + } + dpg.segments = append(dpg.segments, segment) + } + tstb, err := segment.CreateTSTableIfNotExist(shardID) + if err != nil { + return nil, fmt.Errorf("cannot create ts table: %w", err) + } + dpt := &dataPointsInTable{ + timeRange: segment.GetTimeRange(), + tsTable: tstb, + } + dpg.tables = append(dpg.tables, dpt) + return dpt, nil +} + +func (w *writeCallback) handleTagFamily(stm *measure, req *measurev1.WriteRequest) ([]nameValues, []index.Field) { tagFamilies := make([]nameValues, 0, len(stm.schema.TagFamilies)) if len(stm.indexRuleLocators.TagFamilyTRule) != len(stm.GetSchema().GetTagFamilies()) { logger.Panicf("metadata crashed, tag family rule length %d, tag family length %d", @@ -157,18 +212,18 @@ func (w *writeCallback) handle(dst map[string]*dataPointsInGroup, writeEvent *me t.Type, tagValue) r, ok := tfr[t.Name] - if ok || stm.schema.NonTimeSeries { + if ok || stm.schema.IndexMode { fieldKey := index.FieldKey{} switch { case ok: fieldKey.IndexRuleID = r.GetMetadata().GetId() fieldKey.Analyzer = r.Analyzer - case stm.schema.NonTimeSeries: + case stm.schema.IndexMode: fieldKey.TagName = t.Name default: logger.Panicf("metadata crashed, tag family rule %s not found", t.Name) } - toIndex := ok || !stm.schema.NonTimeSeries + toIndex := ok || !stm.schema.IndexMode if encodeTagValue.value != nil { fields = append(fields, index.Field{ Key: fieldKey, @@ -200,57 +255,7 @@ func (w *writeCallback) handle(dst map[string]*dataPointsInGroup, writeEvent *me tagFamilies = append(tagFamilies, tf) } } - dpt.dataPoints.tagFamilies = append(dpt.dataPoints.tagFamilies, tagFamilies) - - if stm.processorManager != nil { - stm.processorManager.onMeasureWrite(uint64(series.ID), &measurev1.InternalWriteRequest{ - Request: &measurev1.WriteRequest{ - Metadata: stm.GetSchema().Metadata, - DataPoint: req.DataPoint, - MessageId: uint64(time.Now().UnixNano()), - }, - EntityValues: writeEvent.EntityValues, - }) - } - - doc := index.Document{ - DocID: uint64(series.ID), - EntityValues: series.Buffer, - Fields: fields, - } - if stm.schema.NonTimeSeries { - doc.Timestamp = ts - } - dpg.docs = append(dpg.docs, doc) - - return dst, nil -} - -func (w *writeCallback) newDpt(tsdb storage.TSDB[*tsTable, option], dpg *dataPointsInGroup, t time.Time, ts int64, shardID common.ShardID) (*dataPointsInTable, error) { - var segment storage.Segment[*tsTable, option] - for _, seg := range dpg.segments { - if seg.GetTimeRange().Contains(ts) { - segment = seg - } - } - if segment == nil { - var err error - segment, err = tsdb.CreateSegmentIfNotExist(t) - if err != nil { - return nil, fmt.Errorf("cannot create segment: %w", err) - } - dpg.segments = append(dpg.segments, segment) - } - tstb, err := segment.CreateTSTableIfNotExist(shardID) - if err != nil { - return nil, fmt.Errorf("cannot create ts table: %w", err) - } - dpt := &dataPointsInTable{ - timeRange: segment.GetTimeRange(), - tsTable: tstb, - } - dpg.tables = append(dpg.tables, dpt) - return dpt, nil + return tagFamilies, fields } func (w *writeCallback) Rev(_ context.Context, message bus.Message) (resp bus.Message) { diff --git a/banyand/stream/stream.go b/banyand/stream/stream.go index 60596f57b..1243a38e4 100644 --- a/banyand/stream/stream.go +++ b/banyand/stream/stream.go @@ -89,7 +89,7 @@ func (s *stream) Close() error { func (s *stream) parseSpec() { s.name, s.group = s.schema.GetMetadata().GetName(), s.schema.GetMetadata().GetGroup() - s.indexRuleLocators, _ = partition.ParseIndexRuleLocators(s.schema.GetEntity(), s.schema.GetTagFamilies(), s.indexRules) + s.indexRuleLocators, _ = partition.ParseIndexRuleLocators(s.schema.GetEntity(), s.schema.GetTagFamilies(), s.indexRules, false) } type streamSpec struct { diff --git a/dist/LICENSE b/dist/LICENSE index a6503207c..f90ac0ca4 100644 --- a/dist/LICENSE +++ b/dist/LICENSE @@ -326,7 +326,7 @@ MIT licenses github.com/go-ole/go-ole v1.3.0 MIT github.com/go-resty/resty/v2 v2.14.0 MIT github.com/go-task/slim-sprig/v3 v3.0.0 MIT - github.com/golang-jwt/jwt/v4 v4.5.0 MIT + github.com/golang-jwt/jwt/v4 v4.5.1 MIT github.com/json-iterator/go v1.1.12 MIT github.com/mattn/go-colorable v0.1.13 MIT github.com/mattn/go-isatty v0.0.20 MIT diff --git a/docs/api-reference.md b/docs/api-reference.md index eb008338c..4954ce1f6 100644 --- a/docs/api-reference.md +++ b/docs/api-reference.md @@ -1076,7 +1076,7 @@ Measure intends to store data point | entity | [Entity](#banyandb-database-v1-Entity) | | entity indicates which tags will be to generate a series and shard a measure | | interval | [string](#string) | | interval indicates how frequently to send a data point valid time units are "ns", "us" (or "µs"), "ms", "s", "m", "h", "d". | | updated_at | [google.protobuf.Timestamp](#google-protobuf-Timestamp) | | updated_at indicates when the measure is updated | -| non_time_series | [bool](#bool) | | non_time_series indicates whether the measure is a time series | +| index_mode | [bool](#bool) | | index_mode specifies whether the data should be stored exclusively in the index, meaning it will not be stored in the data storage system. | diff --git a/go.mod b/go.mod index 56646ccf0..57454f710 100644 --- a/go.mod +++ b/go.mod @@ -83,7 +83,7 @@ require ( github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.3.0 // indirect github.com/gogo/protobuf v1.3.2 // indirect - github.com/golang-jwt/jwt/v4 v4.5.0 // indirect + github.com/golang-jwt/jwt/v4 v4.5.1 // indirect github.com/golang/protobuf v1.5.4 // indirect github.com/google/btree v1.1.3 // indirect github.com/google/pprof v0.0.0-20240910150728-a0b0bb1d4134 // indirect diff --git a/go.sum b/go.sum index d833ec3d6..5c140a3bf 100644 --- a/go.sum +++ b/go.sum @@ -6,8 +6,6 @@ github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAE github.com/RoaringBitmap/roaring v0.9.4/go.mod h1:icnadbWcNyfEHlYdr+tDlOTih1Bf/h+rzPpv4sbomAA= github.com/RoaringBitmap/roaring v1.9.4 h1:yhEIoH4YezLYT04s1nHehNO64EKFTop/wBhxv2QzDdQ= github.com/RoaringBitmap/roaring v1.9.4/go.mod h1:6AXUsoIEzDTFFQCe1RbGA6uFONMhvejWj5rqITANK90= -github.com/SkyAPM/bluge v0.0.0-20241110125856-046bc03b30ab h1:iCQVR0hi6Kd4Pzh/QDHrBIK8TDTKzwEEzrYhdvqoPRg= -github.com/SkyAPM/bluge v0.0.0-20241110125856-046bc03b30ab/go.mod h1:6o9wC3xO3qb5Q7VmD1x0r54qQBDpO9+ghGAQvuOHsCU= github.com/SkyAPM/bluge v0.0.0-20241111124917-c317df1af201 h1:QX/WvtL8j5Zrbs68EVEiOE2nFQSvoT5oTkOFh2uNSpg= github.com/SkyAPM/bluge v0.0.0-20241111124917-c317df1af201/go.mod h1:6o9wC3xO3qb5Q7VmD1x0r54qQBDpO9+ghGAQvuOHsCU= github.com/SkyAPM/clock v1.3.1-0.20220809233656-dc7607c94a97 h1:FKuhJ+6n/DHspGeLleeNbziWnKr9gHKYN4q7NcoCp4s= @@ -104,8 +102,8 @@ github.com/go-task/slim-sprig/v3 v3.0.0/go.mod h1:W848ghGpv3Qj3dhTPRyJypKRiqCdHZ github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/golang-jwt/jwt/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOWzg= -github.com/golang-jwt/jwt/v4 v4.5.0/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= +github.com/golang-jwt/jwt/v4 v4.5.1 h1:JdqV9zKUdtaa9gdPlywC3aeoEsR681PlKC+4F5gQgeo= +github.com/golang-jwt/jwt/v4 v4.5.1/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= diff --git a/pkg/index/index.go b/pkg/index/index.go index 11e6e3795..fc1b8f5c8 100644 --- a/pkg/index/index.go +++ b/pkg/index/index.go @@ -49,9 +49,9 @@ const ( // FieldKey is the key of field in a document. type FieldKey struct { Analyzer string + TagName string SeriesID common.SeriesID IndexRuleID uint32 - TagName string } // Marshal encodes f to string. @@ -221,8 +221,8 @@ func (s Series) SortedField() []byte { // SeriesDocument represents a series document in an index. type SeriesDocument struct { Fields map[string][]byte - Timestamp int64 Key Series + Timestamp int64 } // SeriesStore is an abstract of a series repository. diff --git a/pkg/index/inverted/inverted.go b/pkg/index/inverted/inverted.go index 9a51ed6b6..084ef7290 100644 --- a/pkg/index/inverted/inverted.go +++ b/pkg/index/inverted/inverted.go @@ -58,6 +58,7 @@ var ( defaultUpper = convert.Uint64ToBytes(math.MaxUint64) defaultLower = convert.Uint64ToBytes(0) defaultRangePreloadSize = 1000 + defaultProjection = []string{docIDField} ) // Analyzers is a map that associates each IndexRule_Analyzer type with a corresponding Analyzer. @@ -244,7 +245,7 @@ func (s *store) MatchTerms(field index.Field) (list posting.List, err error) { if err != nil { return nil, err } - iter := newBlugeMatchIterator(documentMatchIterator, reader, nil) + iter := newBlugeMatchIterator(documentMatchIterator, reader, defaultProjection) defer func() { err = multierr.Append(err, iter.Close()) }() @@ -275,7 +276,7 @@ func (s *store) Match(fieldKey index.FieldKey, matches []string, opts *modelv1.C if err != nil { return nil, err } - iter := newBlugeMatchIterator(documentMatchIterator, reader, nil) + iter := newBlugeMatchIterator(documentMatchIterator, reader, defaultProjection) defer func() { err = multierr.Append(err, iter.Close()) }() diff --git a/pkg/index/inverted/inverted_series.go b/pkg/index/inverted/inverted_series.go index 1f245c670..9d0b15edb 100644 --- a/pkg/index/inverted/inverted_series.go +++ b/pkg/index/inverted/inverted_series.go @@ -144,10 +144,6 @@ func (s *store) Search(ctx context.Context, return nil, err } defer func() { - if err := recover(); err != nil { - _ = reader.Close() - panic(err) - } if err := recover(); err != nil { _ = reader.Close() panic(err) @@ -169,13 +165,11 @@ func parseResult(dmi search.DocumentMatchIterator, loadedFields []index.FieldKey return nil, errors.WithMessage(err, "iterate document match iterator") } docIDMap := make(map[uint64]struct{}) - fields := make([]string, 0, len(loadedFields)+3) - fields = append(fields, docIDField, entityField, timestampField) + fields := make([]string, 0, len(loadedFields)) for i := range loadedFields { fields = append(fields, loadedFields[i].Marshal()) } var hitNumber int - ctx := search.NewSearchContext(1, 0) for err == nil && next != nil { hitNumber = next.HitNumber var doc index.SeriesDocument @@ -185,17 +179,8 @@ func parseResult(dmi search.DocumentMatchIterator, loadedFields []index.FieldKey doc.Fields[fields[i]] = nil } } - err = next.LoadDocumentValues(ctx, fields) - if err != nil { - return nil, errors.WithMessagef(err, "visit stored fields, hit: %d", hitNumber) - } - for i := range fields { - vv := next.DocValues(fields[i]) - if vv == nil { - continue - } - value := vv[0] - switch fields[i] { + err = next.VisitStoredFields(func(field string, value []byte) bool { + switch field { case docIDField: id := convert.BytesToUint64(value) if _, ok := docIDMap[id]; !ok { @@ -204,17 +189,15 @@ func parseResult(dmi search.DocumentMatchIterator, loadedFields []index.FieldKey } case entityField: doc.Key.EntityValues = value - case timestampField: - ts, errTime := bluge.DecodeDateTime(value) - if errTime != nil { - return nil, err - } - doc.Timestamp = ts.UnixNano() default: - if _, ok := doc.Fields[fields[i]]; ok { - doc.Fields[fields[i]] = bytes.Clone(value) + if _, ok := doc.Fields[field]; ok { + doc.Fields[field] = bytes.Clone(value) } } + return true + }) + if err != nil { + return nil, errors.WithMessagef(err, "visit stored fields, hit: %d", hitNumber) } if doc.Key.ID > 0 { result = append(result, doc) diff --git a/pkg/index/inverted/sort.go b/pkg/index/inverted/sort.go index 68071cc90..23cc57df2 100644 --- a/pkg/index/inverted/sort.go +++ b/pkg/index/inverted/sort.go @@ -119,7 +119,7 @@ func (si *sortIterator) loadCurrent() bool { size := si.size + si.skipped if size < 0 { // overflow - size = math.MaxInt64 + size = math.MaxInt } topNSearch := bluge.NewTopNSearch(size, si.query.(*queryNode).query).SortBy([]string{si.sortedKey}) if si.skipped > 0 { diff --git a/pkg/partition/index.go b/pkg/partition/index.go index 133dc6362..bbb4daf03 100644 --- a/pkg/partition/index.go +++ b/pkg/partition/index.go @@ -40,7 +40,7 @@ type FieldIndexLocation map[string]map[string]FieldWithType // ParseIndexRuleLocators returns a IndexRuleLocator based on the tag family spec and index rules. func ParseIndexRuleLocators(entity *databasev1.Entity, families []*databasev1.TagFamilySpec, - indexRules []*databasev1.IndexRule, + indexRules []*databasev1.IndexRule, indexMode bool, ) (locators IndexRuleLocator, fil FieldIndexLocation) { locators.EntitySet = make(map[string]struct{}, len(entity.TagNames)) fil = make(FieldIndexLocation) @@ -65,6 +65,9 @@ func ParseIndexRuleLocators(entity *databasev1.Entity, families []*databasev1.Ta if ir != nil { ttr[families[i].Tags[j].Name] = ir } + if ir == nil && !indexMode { + continue + } tagFamily, ok := fil[families[i].Name] if !ok { tagFamily = make(map[string]FieldWithType) diff --git a/pkg/query/model/model.go b/pkg/query/model/model.go index c307918d1..022bcdbc4 100644 --- a/pkg/query/model/model.go +++ b/pkg/query/model/model.go @@ -98,26 +98,6 @@ type MeasureQueryResult interface { Release() } -var ( - BypassResult = &bypassResult{} - dummyResult = &MeasureResult{} -) - -// bypassResult struct. -type bypassResult struct { - // Add fields as necessary -} - -// Implement Pull method. -func (b *bypassResult) Pull() *MeasureResult { - return dummyResult -} - -// Implement Release method. -func (b *bypassResult) Release() { - // No operation -} - // StreamQueryOptions is the options of a stream query. type StreamQueryOptions struct { Name string From b0e2ababad384c08ae8a15a9ea40c77d8607783c Mon Sep 17 00:00:00 2001 From: Gao Hongtao Date: Fri, 15 Nov 2024 09:54:17 +0800 Subject: [PATCH 4/6] Introduce "index_mode" to save data exclusively in the series index Signed-off-by: Gao Hongtao --- CHANGES.md | 1 + banyand/internal/storage/index.go | 64 +- banyand/internal/storage/index_test.go | 4 +- banyand/internal/storage/segment.go | 3 + banyand/internal/storage/storage.go | 6 +- banyand/measure/query.go | 225 +++--- banyand/stream/benchmark_test.go | 6 +- pkg/index/index.go | 28 +- pkg/index/inverted/inverted.go | 50 +- pkg/index/inverted/inverted_series.go | 121 ++-- pkg/index/inverted/inverted_series_test.go | 647 +++++++++++++++++- pkg/index/inverted/query.go | 21 + pkg/pb/v1/series.go | 1 + .../measure/measure_plan_indexscan_local.go | 46 +- .../logical/measure/topn_plan_localscan.go | 3 +- pkg/query/logical/optimizer.go | 3 +- pkg/query/logical/plan.go | 3 + .../stream/stream_plan_indexscan_local.go | 4 +- pkg/query/model/model.go | 24 +- .../testdata/measures/service_traffic.json | 1 + pkg/timestamp/range.go | 16 + .../measure/data/input/index_mode_all.yaml | 23 + .../data/input/index_mode_order_desc.yaml | 26 + .../measure/data/input/index_mode_range.yaml | 30 + .../measure/data/want/index_mode_all.yaml | 107 +++ .../data/want/index_mode_order_desc.yaml | 83 +++ .../measure/data/want/index_mode_range.yaml | 39 ++ test/cases/measure/measure.go | 3 + 28 files changed, 1288 insertions(+), 300 deletions(-) create mode 100644 test/cases/measure/data/input/index_mode_all.yaml create mode 100644 test/cases/measure/data/input/index_mode_order_desc.yaml create mode 100644 test/cases/measure/data/input/index_mode_range.yaml create mode 100644 test/cases/measure/data/want/index_mode_all.yaml create mode 100644 test/cases/measure/data/want/index_mode_order_desc.yaml create mode 100644 test/cases/measure/data/want/index_mode_range.yaml diff --git a/CHANGES.md b/CHANGES.md index 38415e5f1..85b36cba8 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -9,6 +9,7 @@ Release Notes. - Add the `bydbctl analyze series` command to analyze the series data. - Index: Remove sortable field from the stored field. If a field is sortable only, it won't be stored. - Index: Support InsertIfAbsent functionality which ensures documents are only inserted if their docIDs are not already present in the current index. There is a exception for the documents with extra index fields more than the entity's index fields. +- Measure: Introduce "index_mode" to save data exclusively in the series index, ideal for non-timeseries measures. ### Bug Fixes diff --git a/banyand/internal/storage/index.go b/banyand/internal/storage/index.go index ba3bd443f..e25ad7aec 100644 --- a/banyand/internal/storage/index.go +++ b/banyand/internal/storage/index.go @@ -31,6 +31,7 @@ import ( "github.com/apache/skywalking-banyandb/pkg/logger" pbv1 "github.com/apache/skywalking-banyandb/pkg/pb/v1" "github.com/apache/skywalking-banyandb/pkg/query" + "github.com/apache/skywalking-banyandb/pkg/timestamp" ) func (s *segment[T, O]) IndexDB() IndexDB { @@ -38,7 +39,7 @@ func (s *segment[T, O]) IndexDB() IndexDB { } func (s *segment[T, O]) Lookup(ctx context.Context, series []*pbv1.Series) (pbv1.SeriesList, error) { - sl, _, _, err := s.index.filter(ctx, series, nil, nil) + sl, _, _, err := s.index.filter(ctx, series, nil, nil, nil) return sl, err } @@ -76,10 +77,8 @@ func (s *seriesIndex) Write(docs index.Documents) error { }) } -var rangeOpts = index.RangeOpts{} - func (s *seriesIndex) filter(ctx context.Context, series []*pbv1.Series, - projection []index.FieldKey, secondaryQuery index.Query, + projection []index.FieldKey, secondaryQuery index.Query, timeRange *timestamp.TimeRange, ) (sl pbv1.SeriesList, fields FieldResultList, tss []int64, err error) { seriesMatchers := make([]index.SeriesMatcher, len(series)) for i := range series { @@ -88,7 +87,7 @@ func (s *seriesIndex) filter(ctx context.Context, series []*pbv1.Series, return nil, nil, nil, err } } - indexQuery, err := s.store.BuildQuery(seriesMatchers, secondaryQuery) + indexQuery, err := s.store.BuildQuery(seriesMatchers, secondaryQuery, timeRange) if err != nil { return nil, nil, nil, err } @@ -182,7 +181,6 @@ func convertIndexSeriesToSeriesList(indexSeries []index.SeriesDocument, hasField var timestamps []int64 for _, s := range indexSeries { var series pbv1.Series - series.ID = s.Key.ID if err := series.Unmarshal(s.Key.EntityValues); err != nil { return nil, nil, nil, errors.WithMessagef(err, "failed to unmarshal series: %s", s.Key.EntityValues) } @@ -197,11 +195,15 @@ func convertIndexSeriesToSeriesList(indexSeries []index.SeriesDocument, hasField return seriesList, fields, timestamps, nil } -func (s *seriesIndex) Search(ctx context.Context, series []*pbv1.Series, opts IndexSearchOpts) (sl pbv1.SeriesList, frl FieldResultList, tss []int64, err error) { +func (s *seriesIndex) Search(ctx context.Context, series []*pbv1.Series, opts IndexSearchOpts, +) (sl pbv1.SeriesList, frl FieldResultList, tss []int64, sortedValues [][]byte, err error) { tracer := query.GetTracer(ctx) if tracer != nil { var span *query.Span span, ctx = tracer.StartSpan(ctx, "seriesIndex.Search") + if opts.Query != nil { + span.Tagf("secondary_query", "%s", opts.Query.String()) + } defer func() { if err != nil { span.Error(err) @@ -211,19 +213,11 @@ func (s *seriesIndex) Search(ctx context.Context, series []*pbv1.Series, opts In } if opts.Order == nil || opts.Order.Index == nil { - if opts.Query != nil { - sl, frl, tss, err = s.filter(ctx, series, opts.Projection, opts.Query) - } else { - sl, frl, tss, err = s.filter(ctx, series, opts.Projection, nil) - } + sl, frl, tss, err = s.filter(ctx, series, opts.Projection, opts.Query, opts.TimeRange) if err != nil { - return nil, nil, nil, err + return nil, nil, nil, nil, err } - return sl, frl, tss, nil - } - - fieldKey := index.FieldKey{ - IndexRuleID: opts.Order.Index.GetMetadata().Id, + return sl, frl, tss, nil, nil } var span *query.Span if tracer != nil { @@ -240,43 +234,43 @@ func (s *seriesIndex) Search(ctx context.Context, series []*pbv1.Series, opts In for i := range series { seriesMatchers[i], err = convertEntityValuesToSeriesMatcher(series[i]) if err != nil { - return nil, nil, nil, err + return nil, nil, nil, nil, err } } - query, err := s.store.BuildQuery(seriesMatchers, opts.Query) + query, err := s.store.BuildQuery(seriesMatchers, opts.Query, opts.TimeRange) if err != nil { - return nil, nil, nil, err + return nil, nil, nil, nil, err } - iter, err := s.store.SeriesSort(ctx, fieldKey, rangeOpts, - opts.Order.Sort, opts.PreloadSize, query, opts.Projection) + iter, err := s.store.SeriesSort(ctx, query, opts.Order, + opts.PreloadSize, opts.Projection) if err != nil { - return nil, nil, nil, err + return nil, nil, nil, nil, err } defer func() { err = multierr.Append(err, iter.Close()) }() var r int - result := make([]index.SeriesDocument, 0, 10) for iter.Next() { r++ val := iter.Val() - var doc index.SeriesDocument - doc.Fields = maps.Clone(val.Values) - doc.Key.ID = common.SeriesID(val.DocID) - doc.Key.EntityValues = val.EntityValues - result = append(result, doc) - } - sl, frl, tss, err = convertIndexSeriesToSeriesList(result, len(opts.Projection) > 0) - if err != nil { - return nil, nil, nil, errors.WithMessagef(err, "failed to convert index series to series list, matchers: %v, matched: %d", seriesMatchers, len(result)) + var series pbv1.Series + if err = series.Unmarshal(val.EntityValues); err != nil { + return nil, nil, nil, nil, errors.WithMessagef(err, "failed to unmarshal series: %s", val.EntityValues) + } + sl = append(sl, &series) + tss = append(tss, val.Timestamp) + if len(opts.Projection) > 0 { + frl = append(frl, maps.Clone(val.Values)) + } + sortedValues = append(sortedValues, val.SortedValue) } if span != nil { span.Tagf("query", "%s", iter.Query().String()) span.Tagf("rounds", "%d", r) span.Tagf("size", "%d", len(sl)) } - return sl, frl, tss, err + return sl, frl, tss, sortedValues, err } func (s *seriesIndex) Close() error { diff --git a/banyand/internal/storage/index_test.go b/banyand/internal/storage/index_test.go index 6e4db479a..3fd39aed4 100644 --- a/banyand/internal/storage/index_test.go +++ b/banyand/internal/storage/index_test.go @@ -25,6 +25,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "github.com/apache/skywalking-banyandb/api/common" modelv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v1" "github.com/apache/skywalking-banyandb/pkg/index" "github.com/apache/skywalking-banyandb/pkg/logger" @@ -157,11 +158,12 @@ func TestSeriesIndex_Primary(t *testing.T) { seriesQuery.EntityValues = tt.entityValues[i] seriesQueries = append(seriesQueries, &seriesQuery) } - sl, _, _, err := si.filter(ctx, seriesQueries, nil, nil) + sl, _, _, err := si.filter(ctx, seriesQueries, nil, nil, nil) require.NoError(t, err) require.Equal(t, len(tt.entityValues), len(sl)) assert.Equal(t, tt.subject, sl[0].Subject) for i := range tt.expected { + assert.Greater(t, sl[i].ID, common.SeriesID(0)) assert.Equal(t, tt.expected[i][0].GetStr().GetValue(), sl[i].EntityValues[0].GetStr().GetValue()) assert.Equal(t, tt.expected[i][1].GetStr().GetValue(), sl[i].EntityValues[1].GetStr().GetValue()) assert.True(t, sl[0].ID > 0) diff --git a/banyand/internal/storage/segment.go b/banyand/internal/storage/segment.go index 0f7d816b9..02f595814 100644 --- a/banyand/internal/storage/segment.go +++ b/banyand/internal/storage/segment.go @@ -234,6 +234,9 @@ func (sc *segmentController[T, O]) selectSegments(timeRange timestamp.TimeRange) last := len(sc.lst) - 1 for i := range sc.lst { s := sc.lst[last-i] + if s.GetTimeRange().End.Before(timeRange.Start) { + break + } if s.Overlapping(timeRange) { s.incRef() tt = append(tt, s) diff --git a/banyand/internal/storage/storage.go b/banyand/internal/storage/storage.go index 42502515c..aed7a40cd 100644 --- a/banyand/internal/storage/storage.go +++ b/banyand/internal/storage/storage.go @@ -36,7 +36,6 @@ import ( "github.com/apache/skywalking-banyandb/pkg/index" "github.com/apache/skywalking-banyandb/pkg/logger" pbv1 "github.com/apache/skywalking-banyandb/pkg/pb/v1" - "github.com/apache/skywalking-banyandb/pkg/query/model" "github.com/apache/skywalking-banyandb/pkg/timestamp" ) @@ -67,7 +66,8 @@ type SupplyTSDB[T TSTable] func() T // IndexSearchOpts is the options for searching index. type IndexSearchOpts struct { Query index.Query - Order *model.OrderBy + Order *index.OrderBy + TimeRange *timestamp.TimeRange Projection []index.FieldKey PreloadSize int } @@ -81,7 +81,7 @@ type FieldResultList []FieldResult // IndexDB is the interface of index database. type IndexDB interface { Write(docs index.Documents) error - Search(ctx context.Context, series []*pbv1.Series, opts IndexSearchOpts) (pbv1.SeriesList, FieldResultList, []int64, error) + Search(ctx context.Context, series []*pbv1.Series, opts IndexSearchOpts) (pbv1.SeriesList, FieldResultList, []int64, [][]byte, error) } // TSDB allows listing and getting shard details. diff --git a/banyand/measure/query.go b/banyand/measure/query.go index 323993323..e0d074944 100644 --- a/banyand/measure/query.go +++ b/banyand/measure/query.go @@ -18,6 +18,7 @@ package measure import ( + "bytes" "container/heap" "context" "fmt" @@ -145,23 +146,27 @@ func (s *measure) Query(ctx context.Context, mqo model.MeasureQueryOptions) (mqr if mqo.Order == nil { result.ascTS = true - } else if mqo.Order.Sort == modelv1.Sort_SORT_ASC || mqo.Order.Sort == modelv1.Sort_SORT_UNSPECIFIED { - result.ascTS = true - } - switch mqo.OrderByType { - case model.OrderByTypeTime: result.orderByTS = true - case model.OrderByTypeIndex: - result.orderByTS = false - case model.OrderByTypeSeries: - result.orderByTS = false + } else { + if mqo.Order.Sort == modelv1.Sort_SORT_ASC || mqo.Order.Sort == modelv1.Sort_SORT_UNSPECIFIED { + result.ascTS = true + } + switch mqo.Order.Type { + case index.OrderByTypeTime: + result.orderByTS = true + case index.OrderByTypeIndex: + result.orderByTS = false + case index.OrderByTypeSeries: + result.orderByTS = false + } } + return &result, nil } type tagNameWithType struct { - name string - typ pbv1.ValueType + fieldName string + typ pbv1.ValueType } func (s *measure) searchSeriesList(ctx context.Context, series []*pbv1.Series, mqo model.MeasureQueryOptions, @@ -190,8 +195,8 @@ func (s *measure) searchSeriesList(ctx context.Context, series []*pbv1.Series, m if field, ok := fields[n]; ok { indexProjection = append(indexProjection, field.Key) fieldToValueType[field.Key.Marshal()] = tagNameWithType{ - name: n, - typ: field.Type, + fieldName: n, + typ: field.Type, } continue TAG } @@ -205,7 +210,7 @@ func (s *measure) searchSeriesList(ctx context.Context, series []*pbv1.Series, m } seriesFilter := roaring.NewPostingList() for i := range segments { - sll, fieldResultList, _, err := segments[i].IndexDB().Search(ctx, series, storage.IndexSearchOpts{ + sll, fieldResultList, _, _, err := segments[i].IndexDB().Search(ctx, series, storage.IndexSearchOpts{ Query: mqo.Query, Order: mqo.Order, PreloadSize: preloadSize, @@ -239,7 +244,7 @@ func (s *measure) searchSeriesList(ctx context.Context, series []*pbv1.Series, m } for f, v := range fieldResultList[j] { if tnt, ok := fieldToValueType[f]; ok { - tagValues[tnt.name] = mustDecodeTagValue(tnt.typ, v) + tagValues[tnt.fieldName] = mustDecodeTagValue(tnt.typ, v) } else { logger.Panicf("unknown field %s not found in fieldToValueType", f) } @@ -251,13 +256,14 @@ func (s *measure) searchSeriesList(ctx context.Context, series []*pbv1.Series, m func (s *measure) buildIndexQueryResult(ctx context.Context, series []*pbv1.Series, mqo model.MeasureQueryOptions, segments []storage.Segment[*tsTable, option], -) (*indexQueryResult, error) { - r := &indexQueryResult{ - ctx: ctx, - series: series, - mqo: mqo, - segments: segments, - } +) (*indexSortResult, error) { + defer func() { + for i := range segments { + segments[i].DecRef() + } + }() + r := &indexSortResult{} + var indexProjection []index.FieldKey for _, tp := range mqo.TagProjection { tagFamilyLocation := tagFamilyLocation{ name: tp.Family, @@ -266,6 +272,7 @@ func (s *measure) buildIndexQueryResult(ctx context.Context, series []*pbv1.Seri } TAG: for _, n := range tp.Names { + tagFamilyLocation.tagNames = append(tagFamilyLocation.tagNames, n) for i := range s.schema.GetEntity().GetTagNames() { if n == s.schema.GetEntity().GetTagNames()[i] { tagFamilyLocation.projectedEntityOffsets[n] = i @@ -274,10 +281,10 @@ func (s *measure) buildIndexQueryResult(ctx context.Context, series []*pbv1.Seri } if fields, ok := s.fieldIndexLocation[tp.Family]; ok { if field, ok := fields[n]; ok { - r.indexProjection = append(r.indexProjection, field.Key) - tagFamilyLocation.fieldToValueType[field.Key.Marshal()] = tagNameWithType{ - name: n, - typ: field.Type, + indexProjection = append(indexProjection, field.Key) + tagFamilyLocation.fieldToValueType[n] = tagNameWithType{ + fieldName: field.Key.Marshal(), + typ: field.Type, } continue TAG } @@ -286,6 +293,28 @@ func (s *measure) buildIndexQueryResult(ctx context.Context, series []*pbv1.Seri } r.tfl = append(r.tfl, tagFamilyLocation) } + var err error + opts := storage.IndexSearchOpts{ + Query: mqo.Query, + Order: mqo.Order, + PreloadSize: preloadSize, + Projection: indexProjection, + } + + for i := range segments { + if mqo.TimeRange.Include(segments[i].GetTimeRange()) { + opts.TimeRange = nil + } else { + opts.TimeRange = mqo.TimeRange + } + sr := &segResult{} + sr.sll, sr.frl, sr.timestamps, sr.sortedValues, err = segments[i].IndexDB().Search(ctx, series, opts) + if err != nil { + return nil, err + } + r.segResults = append(r.segResults, sr) + } + heap.Init(&r.segResults) return r, nil } @@ -685,88 +714,112 @@ func (qr *queryResult) merge(storedIndexValue map[common.SeriesID]map[string]*mo return result } -var bypassVersions = []int64{0} +var bypassVersions = []int64{1} -type indexQueryResult struct { - ctx context.Context - err error - tfl []tagFamilyLocation - indexProjection []index.FieldKey - series []*pbv1.Series - segments []storage.Segment[*tsTable, option] - sll pbv1.SeriesList - frl storage.FieldResultList - timestamps []int64 - mqo model.MeasureQueryOptions - i int +type indexSortResult struct { + tfl []tagFamilyLocation + segResults segResultHeap } // Pull implements model.MeasureQueryResult. -func (i *indexQueryResult) Pull() *model.MeasureResult { - if i.i < 0 { - if len(i.segments) < 1 { +func (iqr *indexSortResult) Pull() *model.MeasureResult { + if len(iqr.segResults) < 1 { + return nil + } + if len(iqr.segResults) == 1 { + if iqr.segResults[0].i >= len(iqr.segResults[0].sll) { return nil } - i.sll, i.frl, i.timestamps, i.err = i.segments[0].IndexDB().Search(i.ctx, i.series, storage.IndexSearchOpts{ - Query: i.mqo.Query, - Order: i.mqo.Order, - PreloadSize: preloadSize, - Projection: i.indexProjection, - }) - if i.err != nil { - return &model.MeasureResult{ - Error: i.err, - } - } - i.segments = i.segments[1:] - if len(i.sll) < 1 { - return i.Pull() + sr := iqr.segResults[0] + r := iqr.copyTo(sr) + sr.i++ + if sr.i >= len(sr.sll) { + iqr.segResults = iqr.segResults[:0] } - i.i = 0 + return r } - if i.i >= len(i.sll) { - i.i = -1 - return i.Pull() + top := heap.Pop(&iqr.segResults) + sr := top.(*segResult) + r := iqr.copyTo(sr) + sr.i++ + if sr.i < len(sr.sll) { + heap.Push(&iqr.segResults, sr) } - r := &model.MeasureResult{ - SID: i.sll[i.i].ID, - Timestamps: []int64{i.timestamps[i.i]}, + return r +} + +func (iqr *indexSortResult) Release() {} + +func (iqr *indexSortResult) copyTo(src *segResult) (dest *model.MeasureResult) { + index := src.i + dest = &model.MeasureResult{ + SID: src.sll[index].ID, + Timestamps: []int64{src.timestamps[index]}, Versions: bypassVersions, } - for j := range i.tfl { - tagFamily := model.TagFamily{Name: i.tfl[j].name} - for name, offset := range i.tfl[j].projectedEntityOffsets { - tagFamily.Tags = append(tagFamily.Tags, model.Tag{ - Name: name, - Values: []*modelv1.TagValue{i.sll[i.i].EntityValues[offset]}, - }) + for i := range iqr.tfl { + tagFamily := model.TagFamily{Name: iqr.tfl[i].name} + peo := iqr.tfl[i].projectedEntityOffsets + var fr storage.FieldResult + if src.frl != nil { + fr = src.frl[index] } - if i.frl == nil { - continue - } - for f, v := range i.frl[j] { - if tnt, ok := i.tfl[j].fieldToValueType[f]; ok { + for _, n := range iqr.tfl[i].tagNames { + if offset, ok := peo[n]; ok { + tagFamily.Tags = append(tagFamily.Tags, model.Tag{ + Name: n, + Values: []*modelv1.TagValue{src.sll[index].EntityValues[offset]}, + }) + continue + } + if fr == nil { + continue + } + if tnt, ok := iqr.tfl[i].fieldToValueType[n]; ok { tagFamily.Tags = append(tagFamily.Tags, model.Tag{ - Name: tnt.name, - Values: []*modelv1.TagValue{mustDecodeTagValue(tnt.typ, v)}, + Name: n, + Values: []*modelv1.TagValue{mustDecodeTagValue(tnt.typ, fr[tnt.fieldName])}, }) } else { - return &model.MeasureResult{ - Error: errors.Errorf("unknown field %s not found in fieldToValueType", f), - } + logger.Panicf("unknown field %s not found in fieldToValueType", n) } } - r.TagFamilies = append(r.TagFamilies, tagFamily) + dest.TagFamilies = append(dest.TagFamilies, tagFamily) } - i.i++ - return r -} - -func (i *indexQueryResult) Release() { + return dest } type tagFamilyLocation struct { fieldToValueType map[string]tagNameWithType projectedEntityOffsets map[string]int name string + tagNames []string +} + +type segResult struct { + sll pbv1.SeriesList + frl storage.FieldResultList + timestamps []int64 + sortedValues [][]byte + i int +} + +type segResultHeap []*segResult + +func (h segResultHeap) Len() int { return len(h) } +func (h segResultHeap) Less(i, j int) bool { + return bytes.Compare(h[i].sortedValues[h[i].i], h[j].sortedValues[h[j].i]) < 0 +} +func (h segResultHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] } + +func (h *segResultHeap) Push(x interface{}) { + *h = append(*h, x.(*segResult)) +} + +func (h *segResultHeap) Pop() interface{} { + old := *h + n := len(old) + x := old[n-1] + *h = old[0 : n-1] + return x } diff --git a/banyand/stream/benchmark_test.go b/banyand/stream/benchmark_test.go index 0bb418e50..5aa0e10e9 100644 --- a/banyand/stream/benchmark_test.go +++ b/banyand/stream/benchmark_test.go @@ -264,7 +264,7 @@ func generateStream(db storage.TSDB[*tsTable, option]) *stream { } } -func generateStreamQueryOptions(p parameter, index mockIndex) model.StreamQueryOptions { +func generateStreamQueryOptions(p parameter, midx mockIndex) model.StreamQueryOptions { timeRange := timestamp.TimeRange{ Start: time.Unix(int64(p.startTimestamp), 0), End: time.Unix(int64(p.endTimestamp), 0), @@ -287,7 +287,7 @@ func generateStreamQueryOptions(p parameter, index mockIndex) model.StreamQueryO num := generateRandomNumber(int64(p.tagCardinality)) value := filterTagValuePrefix + strconv.Itoa(num) filter := mockFilter{ - index: index, + index: midx, value: value, } indexRule := &databasev1.IndexRule{ @@ -297,7 +297,7 @@ func generateStreamQueryOptions(p parameter, index mockIndex) model.StreamQueryO Tags: []string{"filter-tag"}, Type: databasev1.IndexRule_TYPE_INVERTED, } - order := &model.OrderBy{ + order := &index.OrderBy{ Index: indexRule, Sort: modelv1.Sort_SORT_ASC, } diff --git a/pkg/index/index.go b/pkg/index/index.go index fc1b8f5c8..88485605d 100644 --- a/pkg/index/index.go +++ b/pkg/index/index.go @@ -174,7 +174,7 @@ type Writer interface { // FieldIterable allows building a FieldIterator. type FieldIterable interface { - BuildQuery(seriesMatchers []SeriesMatcher, secondaryQuery Query) (Query, error) + BuildQuery(seriesMatchers []SeriesMatcher, secondaryQuery Query, timeRange *timestamp.TimeRange) (Query, error) Iterator(ctx context.Context, fieldKey FieldKey, termRange RangeOpts, order modelv1.Sort, preLoadSize int) (iter FieldIterator[*DocumentResult], err error) Sort(ctx context.Context, sids []common.SeriesID, fieldKey FieldKey, @@ -206,11 +206,10 @@ type Store interface { // Series represents a series in an index. type Series struct { EntityValues []byte - ID common.SeriesID } func (s Series) String() string { - return fmt.Sprintf("%s:%d", s.EntityValues, s.ID) + return convert.BytesToString(s.EntityValues) } // SortedField returns the value of the sorted field. @@ -225,14 +224,33 @@ type SeriesDocument struct { Timestamp int64 } +// OrderByType is the type of order by. +type OrderByType int + +const ( + // OrderByTypeTime is the order by time. + OrderByTypeTime OrderByType = iota + // OrderByTypeIndex is the order by index. + OrderByTypeIndex + // OrderByTypeSeries is the order by series. + OrderByTypeSeries +) + +// OrderBy is the order by rule. +type OrderBy struct { + Index *databasev1.IndexRule + Sort modelv1.Sort + Type OrderByType +} + // SeriesStore is an abstract of a series repository. type SeriesStore interface { Store // Search returns a list of series that match the given matchers. Search(context.Context, []FieldKey, Query) ([]SeriesDocument, error) SeriesIterator(context.Context) (FieldIterator[Series], error) - SeriesSort(ctx context.Context, fieldKey FieldKey, termRange RangeOpts, order modelv1.Sort, - preLoadSize int, query Query, fieldKeys []FieldKey) (iter FieldIterator[*DocumentResult], err error) + SeriesSort(ctx context.Context, indexQuery Query, orderBy *OrderBy, + preLoadSize int, fieldKeys []FieldKey) (iter FieldIterator[*DocumentResult], err error) } // SeriesMatcherType represents the type of series matcher. diff --git a/pkg/index/inverted/inverted.go b/pkg/index/inverted/inverted.go index 084ef7290..4772f8b89 100644 --- a/pkg/index/inverted/inverted.go +++ b/pkg/index/inverted/inverted.go @@ -48,10 +48,8 @@ import ( const ( docIDField = "_id" batchSize = 1024 - seriesIDField = "series_id" - entityField = "entity" - idField = "id" - timestampField = "timestamp" + seriesIDField = "_series_id" + timestampField = "_timestamp" ) var ( @@ -317,22 +315,24 @@ func (s *store) Range(fieldKey index.FieldKey, opts index.RangeOpts) (list posti } type blugeMatchIterator struct { - delegated search.DocumentMatchIterator - err error - closer io.Closer - ctx *search.Context - current index.DocumentResult - hit int + delegated search.DocumentMatchIterator + err error + closer io.Closer + ctx *search.Context + loadDocValues []string + current index.DocumentResult + hit int } func newBlugeMatchIterator(delegated search.DocumentMatchIterator, closer io.Closer, - _ []string, + loadDocValues []string, ) blugeIterator { bmi := &blugeMatchIterator{ - delegated: delegated, - closer: closer, - current: index.DocumentResult{}, - ctx: search.NewSearchContext(1, 0), + delegated: delegated, + closer: closer, + current: index.DocumentResult{}, + ctx: search.NewSearchContext(1, 0), + loadDocValues: loadDocValues, } return bmi } @@ -359,9 +359,23 @@ func (bmi *blugeMatchIterator) Next() bool { if len(match.SortValue) > 0 { bmi.current.SortedValue = match.SortValue[0] } - err := match.VisitStoredFields(bmi.setVal) - bmi.err = multierr.Combine(bmi.err, err) - return bmi.err == nil + if len(bmi.loadDocValues) == 0 { + err := match.VisitStoredFields(bmi.setVal) + bmi.err = multierr.Combine(bmi.err, err) + return bmi.err == nil + } + if err := match.LoadDocumentValues(bmi.ctx, bmi.loadDocValues); err != nil { + bmi.err = multierr.Combine(bmi.err, err) + return false + } + for _, dv := range bmi.loadDocValues { + vv := match.DocValues(dv) + if len(vv) == 0 { + continue + } + bmi.setVal(dv, vv[0]) + } + return true } func (bmi *blugeMatchIterator) setVal(field string, value []byte) bool { diff --git a/pkg/index/inverted/inverted_series.go b/pkg/index/inverted/inverted_series.go index 9d0b15edb..59feb2c50 100644 --- a/pkg/index/inverted/inverted_series.go +++ b/pkg/index/inverted/inverted_series.go @@ -30,10 +30,10 @@ import ( "github.com/pkg/errors" "go.uber.org/multierr" - "github.com/apache/skywalking-banyandb/api/common" modelv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v1" "github.com/apache/skywalking-banyandb/pkg/convert" "github.com/apache/skywalking-banyandb/pkg/index" + "github.com/apache/skywalking-banyandb/pkg/timestamp" ) var emptySeries = make([]index.SeriesDocument, 0) @@ -49,12 +49,15 @@ func (s *store) SeriesBatch(batch index.Batch) error { b := generateBatch() defer releaseBatch(b) for _, d := range batch.Documents { - doc := bluge.NewDocument(convert.BytesToString(convert.Uint64ToBytes(d.DocID))) + doc := bluge.NewDocument(convert.BytesToString(d.EntityValues)) for _, f := range d.Fields { tf := bluge.NewKeywordFieldBytes(f.Key.Marshal(), f.Term) - if !f.NoSort { + if !f.Index { + tf.FieldOptions = 0 + } else if !f.NoSort { tf.Sortable() } + if f.Store { tf.StoreValue() } @@ -64,7 +67,6 @@ func (s *store) SeriesBatch(batch index.Batch) error { doc.AddField(tf) } - doc.AddField(bluge.NewKeywordFieldBytes(entityField, d.EntityValues).StoreValue()) if d.Timestamp > 0 { doc.AddField(bluge.NewDateTimeField(timestampField, time.Unix(0, d.Timestamp)).StoreValue()) } @@ -78,7 +80,7 @@ func (s *store) SeriesBatch(batch index.Batch) error { } // BuildQuery implements index.SeriesStore. -func (s *store) BuildQuery(seriesMatchers []index.SeriesMatcher, secondaryQuery index.Query) (index.Query, error) { +func (s *store) BuildQuery(seriesMatchers []index.SeriesMatcher, secondaryQuery index.Query, timeRange *timestamp.TimeRange) (index.Query, error) { if len(seriesMatchers) == 0 { return secondaryQuery, nil } @@ -90,19 +92,19 @@ func (s *store) BuildQuery(seriesMatchers []index.SeriesMatcher, secondaryQuery case index.SeriesMatcherTypeExact: match := convert.BytesToString(seriesMatchers[i].Match) q := bluge.NewTermQuery(match) - q.SetField(entityField) + q.SetField(docIDField) qs[i] = q nodes = append(nodes, newTermNode(match, nil)) case index.SeriesMatcherTypePrefix: match := convert.BytesToString(seriesMatchers[i].Match) q := bluge.NewPrefixQuery(match) - q.SetField(entityField) + q.SetField(docIDField) qs[i] = q nodes = append(nodes, newPrefixNode(match)) case index.SeriesMatcherTypeWildcard: match := convert.BytesToString(seriesMatchers[i].Match) q := bluge.NewWildcardQuery(match) - q.SetField(entityField) + q.SetField(docIDField) qs[i] = q nodes = append(nodes, newWildcardNode(match)) default: @@ -132,6 +134,12 @@ func (s *store) BuildQuery(seriesMatchers []index.SeriesMatcher, secondaryQuery query.AddMust(secondaryQuery.(*queryNode).query) node.Append(secondaryQuery.(*queryNode).node) } + if timeRange != nil { + q := bluge.NewDateRangeInclusiveQuery(timeRange.Start, timeRange.End, timeRange.IncludeStart, timeRange.IncludeEnd) + q.SetField(timestampField) + query.AddMust(q) + node.Append(newTimeRangeNode(timeRange)) + } return &queryNode{query, node}, nil } @@ -164,7 +172,6 @@ func parseResult(dmi search.DocumentMatchIterator, loadedFields []index.FieldKey if err != nil { return nil, errors.WithMessage(err, "iterate document match iterator") } - docIDMap := make(map[uint64]struct{}) fields := make([]string, 0, len(loadedFields)) for i := range loadedFields { fields = append(fields, loadedFields[i].Marshal()) @@ -179,16 +186,19 @@ func parseResult(dmi search.DocumentMatchIterator, loadedFields []index.FieldKey doc.Fields[fields[i]] = nil } } + var errTime error err = next.VisitStoredFields(func(field string, value []byte) bool { switch field { case docIDField: - id := convert.BytesToUint64(value) - if _, ok := docIDMap[id]; !ok { - doc.Key.ID = common.SeriesID(convert.BytesToUint64(value)) - docIDMap[id] = struct{}{} - } - case entityField: doc.Key.EntityValues = value + case timestampField: + var ts time.Time + ts, errTime = bluge.DecodeDateTime(value) + if errTime != nil { + err = errTime + return false + } + doc.Timestamp = ts.UnixNano() default: if _, ok := doc.Fields[field]; ok { doc.Fields[field] = bytes.Clone(value) @@ -196,10 +206,10 @@ func parseResult(dmi search.DocumentMatchIterator, loadedFields []index.FieldKey } return true }) - if err != nil { + if err = multierr.Combine(err, errTime); err != nil { return nil, errors.WithMessagef(err, "visit stored fields, hit: %d", hitNumber) } - if doc.Key.ID > 0 { + if len(doc.Key.EntityValues) > 0 { result = append(result, doc) } next, err = dmi.Next() @@ -210,58 +220,39 @@ func parseResult(dmi search.DocumentMatchIterator, loadedFields []index.FieldKey return result, nil } -func (s *store) SeriesSort(ctx context.Context, fieldKey index.FieldKey, termRange index.RangeOpts, order modelv1.Sort, - preLoadSize int, indexQuery index.Query, fieldKeys []index.FieldKey, +func (s *store) SeriesSort(ctx context.Context, indexQuery index.Query, orderBy *index.OrderBy, + preLoadSize int, fieldKeys []index.FieldKey, ) (iter index.FieldIterator[*index.DocumentResult], err error) { - if termRange.Lower != nil && - termRange.Upper != nil && - bytes.Compare(termRange.Lower, termRange.Upper) > 0 { - return index.DummyFieldIterator, nil + var sortedKey string + switch orderBy.Type { + case index.OrderByTypeTime: + sortedKey = timestampField + case index.OrderByTypeIndex: + fieldKey := index.FieldKey{ + IndexRuleID: orderBy.Index.Metadata.Id, + } + sortedKey = fieldKey.Marshal() + default: + return nil, errors.Errorf("unsupported order by type: %v", orderBy.Type) + } + if orderBy.Sort == modelv1.Sort_SORT_DESC { + sortedKey = "-" + sortedKey } + fields := make([]string, 0, len(fieldKeys)) + for i := range fieldKeys { + fields = append(fields, fieldKeys[i].Marshal()) + } + if !s.closer.AddRunning() { return nil, nil } - reader, err := s.writer.Reader() if err != nil { return nil, err } - fk := fieldKey.Marshal() - if termRange.Upper == nil { - termRange.Upper = defaultUpper - } - if termRange.Lower == nil { - termRange.Lower = defaultLower - } - rangeQuery := bluge.NewBooleanQuery() - rangeQuery.AddMust(bluge.NewTermRangeInclusiveQuery( - string(termRange.Lower), - string(termRange.Upper), - termRange.IncludesLower, - termRange.IncludesUpper, - ). - SetField(fk)) - rangeNode := newMustNode() - rangeNode.Append(newTermRangeInclusiveNode(string(termRange.Lower), string(termRange.Upper), termRange.IncludesLower, termRange.IncludesUpper, nil)) - - sortedKey := fk - if order == modelv1.Sort_SORT_DESC { - sortedKey = "-" + sortedKey - } - query := bluge.NewBooleanQuery().AddMust(rangeQuery) - node := newMustNode() - node.Append(rangeNode) - if indexQuery != nil && indexQuery.(*queryNode).query != nil { - query.AddMust(indexQuery.(*queryNode).query) - node.Append(indexQuery.(*queryNode).node) - } - fields := make([]string, 0, len(fieldKeys)) - for i := range fieldKeys { - fields = append(fields, fieldKeys[i].Marshal()) - } - result := &sortIterator{ - query: &queryNode{query, node}, + return &sortIterator{ + query: indexQuery, fields: fields, reader: reader, sortedKey: sortedKey, @@ -269,13 +260,11 @@ func (s *store) SeriesSort(ctx context.Context, fieldKey index.FieldKey, termRan closer: s.closer, ctx: ctx, newIterator: newSeriesIterator, - } - return result, nil + }, nil } type seriesIterator struct { *blugeMatchIterator - needToLoadFields []string } func newSeriesIterator(delegated search.DocumentMatchIterator, closer io.Closer, @@ -288,7 +277,6 @@ func newSeriesIterator(delegated search.DocumentMatchIterator, closer io.Closer, ctx: search.NewSearchContext(1, 0), current: index.DocumentResult{Values: make(map[string][]byte, len(needToLoadFields))}, }, - needToLoadFields: append(needToLoadFields, entityField, docIDField, seriesIDField, timestampField), } for _, f := range needToLoadFields { si.current.Values[f] = nil @@ -312,7 +300,6 @@ func (si *seriesIterator) Next() bool { si.current.Values[i] = nil } si.current.DocID = 0 - si.current.SeriesID = 0 si.current.Timestamp = 0 si.current.SortedValue = nil if len(match.SortValue) > 0 { @@ -329,10 +316,8 @@ func (si *seriesIterator) Next() bool { func (si *seriesIterator) setVal(field string, value []byte) bool { switch field { - case entityField: - si.current.EntityValues = value case docIDField: - si.current.DocID = convert.BytesToUint64(value) + si.current.EntityValues = value case timestampField: ts, errTime := bluge.DecodeDateTime(value) if errTime != nil { @@ -356,7 +341,7 @@ func (s *store) SeriesIterator(ctx context.Context) (index.FieldIterator[index.S defer func() { _ = reader.Close() }() - dict, err := reader.DictionaryIterator(entityField, nil, nil, nil) + dict, err := reader.DictionaryIterator(docIDField, nil, nil, nil) if err != nil { return nil, err } diff --git a/pkg/index/inverted/inverted_series_test.go b/pkg/index/inverted/inverted_series_test.go index cd0a3bd46..ac546ca9f 100644 --- a/pkg/index/inverted/inverted_series_test.go +++ b/pkg/index/inverted/inverted_series_test.go @@ -19,15 +19,22 @@ package inverted import ( "context" + "maps" "testing" + "time" + "github.com/blugelabs/bluge" + "github.com/blugelabs/bluge/numeric" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/apache/skywalking-banyandb/api/common" + commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1" + databasev1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1" + modelv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v1" "github.com/apache/skywalking-banyandb/pkg/convert" "github.com/apache/skywalking-banyandb/pkg/index" "github.com/apache/skywalking-banyandb/pkg/logger" + "github.com/apache/skywalking-banyandb/pkg/timestamp" ) var ( @@ -37,6 +44,9 @@ var ( fieldKeyServiceName = index.FieldKey{ IndexRuleID: 6, } + fieldKeyStartTime = index.FieldKey{ + IndexRuleID: 21, + } ) func TestStore_Search(t *testing.T) { @@ -66,7 +76,6 @@ func TestStore_Search(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(1), EntityValues: []byte("test1"), }, }, @@ -78,33 +87,32 @@ func TestStore_Search(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(2), EntityValues: []byte("test2"), }, Fields: map[string][]byte{ fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(100)), fieldKeyServiceName.Marshal(): []byte("svc2"), }, + Timestamp: int64(101), }, { Key: index.Series{ - ID: common.SeriesID(3), - EntityValues: []byte("test3"), + EntityValues: []byte("test1"), }, Fields: map[string][]byte{ - fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(500)), + fieldKeyDuration.Marshal(): nil, fieldKeyServiceName.Marshal(): nil, }, }, { Key: index.Series{ - ID: common.SeriesID(1), - EntityValues: []byte("test1"), + EntityValues: []byte("test3"), }, Fields: map[string][]byte{ - fieldKeyDuration.Marshal(): nil, + fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(500)), fieldKeyServiceName.Marshal(): nil, }, + Timestamp: int64(1001), }, }, }, @@ -114,30 +122,30 @@ func TestStore_Search(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(2), EntityValues: []byte("test2"), }, Fields: map[string][]byte{ fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(100)), }, + Timestamp: int64(101), }, + { Key: index.Series{ - ID: common.SeriesID(3), - EntityValues: []byte("test3"), + EntityValues: []byte("test1"), }, Fields: map[string][]byte{ - fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(500)), + fieldKeyDuration.Marshal(): nil, }, }, { Key: index.Series{ - ID: common.SeriesID(1), - EntityValues: []byte("test1"), + EntityValues: []byte("test3"), }, Fields: map[string][]byte{ - fieldKeyDuration.Marshal(): nil, + fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(500)), }, + Timestamp: int64(1001), }, }, }, @@ -147,17 +155,16 @@ func TestStore_Search(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(2), EntityValues: []byte("test2"), }, Fields: map[string][]byte{ fieldKeyServiceName.Marshal(): []byte("svc2"), }, + Timestamp: int64(101), }, { Key: index.Series{ - ID: common.SeriesID(3), - EntityValues: []byte("test3"), + EntityValues: []byte("test1"), }, Fields: map[string][]byte{ fieldKeyServiceName.Marshal(): nil, @@ -165,12 +172,12 @@ func TestStore_Search(t *testing.T) { }, { Key: index.Series{ - ID: common.SeriesID(1), - EntityValues: []byte("test1"), + EntityValues: []byte("test3"), }, Fields: map[string][]byte{ fieldKeyServiceName.Marshal(): nil, }, + Timestamp: int64(1001), }, }, }, @@ -191,7 +198,7 @@ func TestStore_Search(t *testing.T) { name += string(term) + "-" } t.Run(name, func(t *testing.T) { - query, err := s.BuildQuery(matchers, nil) + query, err := s.BuildQuery(matchers, nil, nil) require.NotEmpty(t, query.String()) require.NoError(t, err) got, err := s.Search(context.Background(), tt.projection, query) @@ -228,21 +235,26 @@ func TestStore_SearchWildcard(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(1), EntityValues: []byte("test1"), }, }, { Key: index.Series{ - ID: common.SeriesID(2), EntityValues: []byte("test2"), }, + Timestamp: int64(101), }, { Key: index.Series{ - ID: common.SeriesID(3), EntityValues: []byte("test3"), }, + Timestamp: int64(1001), + }, + { + Key: index.Series{ + EntityValues: []byte("test4"), + }, + Timestamp: int64(2001), }, }, }, @@ -251,9 +263,9 @@ func TestStore_SearchWildcard(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(2), EntityValues: []byte("test2"), }, + Timestamp: int64(101), }, }, }, @@ -262,7 +274,6 @@ func TestStore_SearchWildcard(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(1), EntityValues: []byte("test1"), }, }, @@ -281,7 +292,7 @@ func TestStore_SearchWildcard(t *testing.T) { Type: index.SeriesMatcherTypeWildcard, Match: tt.wildcard, }, - }, nil) + }, nil, nil) require.NoError(t, err) require.NotEmpty(t, query.String()) got, err := s.Search(context.Background(), tt.projection, query) @@ -318,21 +329,26 @@ func TestStore_SearchPrefix(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(1), EntityValues: []byte("test1"), }, }, { Key: index.Series{ - ID: common.SeriesID(2), EntityValues: []byte("test2"), }, + Timestamp: int64(101), }, { Key: index.Series{ - ID: common.SeriesID(3), EntityValues: []byte("test3"), }, + Timestamp: int64(1001), + }, + { + Key: index.Series{ + EntityValues: []byte("test4"), + }, + Timestamp: int64(2001), }, }, }, @@ -349,7 +365,7 @@ func TestStore_SearchPrefix(t *testing.T) { Type: index.SeriesMatcherTypePrefix, Match: tt.prefix, }, - }, nil) + }, nil, nil) require.NoError(t, err) require.NotEmpty(t, query.String()) got, err := s.Search(context.Background(), tt.projection, query) @@ -359,41 +375,600 @@ func TestStore_SearchPrefix(t *testing.T) { } } +func TestStore_SearchWithSecondaryQuery(t *testing.T) { + tester := require.New(t) + path, fn := setUp(tester) + s, err := NewStore(StoreOpts{ + Path: path, + Logger: logger.GetLogger("test"), + }) + tester.NoError(err) + defer func() { + tester.NoError(s.Close()) + fn() + }() + + // Setup some data + setupData(tester, s) + + // Define the secondary query + secondaryQuery := &queryNode{ + query: bluge.NewTermQuery("svc2").SetField(fieldKeyServiceName.Marshal()), + node: newTermNode("svc2", nil), + } + + // Test cases + tests := []struct { + term [][]byte + want []index.SeriesDocument + projection []index.FieldKey + }{ + { + term: [][]byte{[]byte("test2")}, + projection: []index.FieldKey{fieldKeyServiceName, fieldKeyDuration, {TagName: "short_name"}}, + want: []index.SeriesDocument{ + { + Key: index.Series{ + EntityValues: []byte("test2"), + }, + Fields: map[string][]byte{ + fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(100)), + fieldKeyServiceName.Marshal(): []byte("svc2"), + "short_name": []byte("t2"), + }, + Timestamp: int64(101), + }, + }, + }, + { + term: [][]byte{[]byte("test3")}, + projection: []index.FieldKey{fieldKeyServiceName, fieldKeyDuration}, + want: []index.SeriesDocument{}, + }, + { + term: [][]byte{[]byte("test1")}, + projection: []index.FieldKey{fieldKeyServiceName, fieldKeyDuration}, + want: []index.SeriesDocument{}, + }, + { + term: [][]byte{[]byte("test2"), []byte("test3")}, + projection: []index.FieldKey{fieldKeyServiceName, fieldKeyDuration, {TagName: "short_name"}}, + want: []index.SeriesDocument{ + { + Key: index.Series{ + EntityValues: []byte("test2"), + }, + Fields: map[string][]byte{ + fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(100)), + fieldKeyServiceName.Marshal(): []byte("svc2"), + "short_name": []byte("t2"), + }, + Timestamp: int64(101), + }, + }, + }, + { + term: [][]byte{[]byte("test1"), []byte("test2")}, + projection: []index.FieldKey{fieldKeyServiceName, fieldKeyDuration}, + want: []index.SeriesDocument{ + { + Key: index.Series{ + EntityValues: []byte("test2"), + }, + Fields: map[string][]byte{ + fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(100)), + fieldKeyServiceName.Marshal(): []byte("svc2"), + }, + Timestamp: int64(101), + }, + }, + }, + } + + for _, tt := range tests { + var matchers []index.SeriesMatcher + var name string + for _, term := range tt.term { + matchers = append(matchers, index.SeriesMatcher{ + Type: index.SeriesMatcherTypeExact, + Match: term, + }) + name += string(term) + "-" + } + t.Run(name, func(t *testing.T) { + query, err := s.BuildQuery(matchers, secondaryQuery, nil) + require.NotEmpty(t, query.String()) + require.NoError(t, err) + got, err := s.Search(context.Background(), tt.projection, query) + require.NoError(t, err) + assert.Equal(t, tt.want, got) + }) + } +} + +func TestStore_SeriesSort(t *testing.T) { + tester := require.New(t) + path, fn := setUp(tester) + s, err := NewStore(StoreOpts{ + Path: path, + Logger: logger.GetLogger("test"), + }) + tester.NoError(err) + defer func() { + tester.NoError(s.Close()) + fn() + }() + + // Setup some data + setupData(tester, s) + + // Define the order by field + orderBy := &index.OrderBy{ + Index: &databasev1.IndexRule{ + Metadata: &commonv1.Metadata{ + Id: fieldKeyStartTime.IndexRuleID, + }, + }, + Sort: modelv1.Sort_SORT_ASC, + Type: index.OrderByTypeIndex, + } + + // Test cases + tests := []struct { + name string + orderBy *index.OrderBy + timeRange *timestamp.TimeRange + want []index.DocumentResult + fieldKeys []index.FieldKey + }{ + { + name: "Sort by start_time ascending", + orderBy: orderBy, + fieldKeys: []index.FieldKey{fieldKeyStartTime}, + want: []index.DocumentResult{ + { + EntityValues: []byte("test2"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(100)), + }, + SortedValue: convert.Int64ToBytes(int64(100)), + }, + { + EntityValues: []byte("test3"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(1000)), + }, + SortedValue: convert.Int64ToBytes(int64(1000)), + }, + { + EntityValues: []byte("test4"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(2000)), + }, + SortedValue: convert.Int64ToBytes(int64(2000)), + }, + { + EntityValues: []byte("test1"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): nil, + }, + SortedValue: []byte{0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff}, + }, + }, + }, + { + name: "Sort by start_time descending", + orderBy: &index.OrderBy{ + Index: &databasev1.IndexRule{ + Metadata: &commonv1.Metadata{ + Id: fieldKeyStartTime.IndexRuleID, + }, + }, + Sort: modelv1.Sort_SORT_DESC, + Type: index.OrderByTypeIndex, + }, + fieldKeys: []index.FieldKey{fieldKeyStartTime}, + want: []index.DocumentResult{ + { + EntityValues: []byte("test4"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(2000)), + }, + SortedValue: convert.Int64ToBytes(int64(2000)), + }, + { + EntityValues: []byte("test3"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(1000)), + }, + SortedValue: convert.Int64ToBytes(int64(1000)), + }, + { + EntityValues: []byte("test2"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(100)), + }, + SortedValue: convert.Int64ToBytes(int64(100)), + }, + { + EntityValues: []byte("test1"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): nil, + }, + SortedValue: []byte{0x00}, + }, + }, + }, + { + name: "Sort by start_time ascending with time range 100 to 1000", + orderBy: orderBy, + fieldKeys: []index.FieldKey{fieldKeyStartTime}, + timeRange: func() *timestamp.TimeRange { + tr := timestamp.NewInclusiveTimeRange(time.Unix(0, 100), time.Unix(0, 1000)) + return &tr + }(), + want: []index.DocumentResult{ + { + EntityValues: []byte("test2"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(100)), + }, + SortedValue: convert.Int64ToBytes(int64(100)), + }, + { + EntityValues: []byte("test3"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(1000)), + }, + SortedValue: convert.Int64ToBytes(int64(1000)), + }, + }, + }, + { + name: "Sort by start_time ascending with time range 0 to 2000", + orderBy: orderBy, + fieldKeys: []index.FieldKey{fieldKeyStartTime}, + timeRange: func() *timestamp.TimeRange { + tr := timestamp.NewInclusiveTimeRange(time.Unix(0, 0), time.Unix(0, 2000)) + return &tr + }(), + want: []index.DocumentResult{ + { + EntityValues: []byte("test2"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(100)), + }, + SortedValue: convert.Int64ToBytes(int64(100)), + }, + { + EntityValues: []byte("test3"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(1000)), + }, + SortedValue: convert.Int64ToBytes(int64(1000)), + }, + { + EntityValues: []byte("test4"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(2000)), + }, + SortedValue: convert.Int64ToBytes(int64(2000)), + }, + }, + }, + { + name: "Sort by start_time ascending with time range 500 to 1500", + orderBy: orderBy, + fieldKeys: []index.FieldKey{fieldKeyStartTime}, + timeRange: func() *timestamp.TimeRange { + tr := timestamp.NewInclusiveTimeRange(time.Unix(0, 500), time.Unix(0, 1500)) + return &tr + }(), + want: []index.DocumentResult{ + { + EntityValues: []byte("test3"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(1000)), + }, + SortedValue: convert.Int64ToBytes(int64(1000)), + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + var secondaryQuery index.Query + if tt.timeRange != nil { + secondaryQuery = &queryNode{ + query: bluge.NewTermRangeInclusiveQuery( + string(convert.Int64ToBytes(tt.timeRange.Start.Local().UnixNano())), + string(convert.Int64ToBytes(tt.timeRange.End.Local().UnixNano())), + tt.timeRange.IncludeStart, + tt.timeRange.IncludeEnd, + ).SetField(fieldKeyStartTime.Marshal()), + } + } + query, err := s.BuildQuery([]index.SeriesMatcher{ + { + Type: index.SeriesMatcherTypePrefix, + Match: []byte("test"), + }, + }, secondaryQuery, nil) + require.NoError(t, err) + iter, err := s.SeriesSort(context.Background(), query, tt.orderBy, 10, tt.fieldKeys) + require.NoError(t, err) + defer iter.Close() + + var got []index.DocumentResult + for iter.Next() { + var g index.DocumentResult + val := iter.Val() + g.DocID = val.DocID + g.EntityValues = val.EntityValues + g.Values = maps.Clone(val.Values) + g.SortedValue = val.SortedValue + got = append(got, g) + } + assert.Equal(t, tt.want, got) + }) + } +} + +func TestStore_TimestampSort(t *testing.T) { + tester := require.New(t) + path, fn := setUp(tester) + s, err := NewStore(StoreOpts{ + Path: path, + Logger: logger.GetLogger("test"), + }) + tester.NoError(err) + defer func() { + tester.NoError(s.Close()) + fn() + }() + + // Setup some data + setupData(tester, s) + + // Define the order by field + orderBy := &index.OrderBy{ + Type: index.OrderByTypeTime, + Sort: modelv1.Sort_SORT_ASC, + } + + // Test cases + tests := []struct { + name string + orderBy *index.OrderBy + timeRange *timestamp.TimeRange + want []index.DocumentResult + fieldKeys []index.FieldKey + }{ + { + name: "Sort by timestamp ascending", + orderBy: orderBy, + want: []index.DocumentResult{ + { + EntityValues: []byte("test2"), + Timestamp: int64(101), + SortedValue: numeric.MustNewPrefixCodedInt64(101, 0), + }, + { + EntityValues: []byte("test3"), + Timestamp: int64(1001), + SortedValue: numeric.MustNewPrefixCodedInt64(1001, 0), + }, + { + EntityValues: []byte("test4"), + Timestamp: int64(2001), + SortedValue: numeric.MustNewPrefixCodedInt64(2001, 0), + }, + { + EntityValues: []byte("test1"), + Timestamp: 0, + SortedValue: []byte{0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff}, + }, + }, + }, + { + name: "Sort by timestamp descending", + orderBy: &index.OrderBy{ + Type: index.OrderByTypeTime, + Sort: modelv1.Sort_SORT_DESC, + }, + want: []index.DocumentResult{ + { + EntityValues: []byte("test4"), + Timestamp: int64(2001), + SortedValue: numeric.MustNewPrefixCodedInt64(2001, 0), + }, + { + EntityValues: []byte("test3"), + Timestamp: int64(1001), + SortedValue: numeric.MustNewPrefixCodedInt64(1001, 0), + }, + { + EntityValues: []byte("test2"), + Timestamp: int64(101), + SortedValue: numeric.MustNewPrefixCodedInt64(101, 0), + }, + { + EntityValues: []byte("test1"), + Timestamp: 0, + SortedValue: []byte{0x00}, + }, + }, + }, + { + name: "Sort by timestamp ascending with time range 100 to 1000", + orderBy: orderBy, + timeRange: func() *timestamp.TimeRange { + tr := timestamp.NewInclusiveTimeRange(time.Unix(0, 100), time.Unix(0, 1000)) + return &tr + }(), + want: []index.DocumentResult{ + { + EntityValues: []byte("test2"), + Timestamp: int64(101), + SortedValue: numeric.MustNewPrefixCodedInt64(101, 0), + }, + }, + }, + { + name: "Sort by timestamp ascending with time range 0 to 2000", + orderBy: orderBy, + timeRange: func() *timestamp.TimeRange { + tr := timestamp.NewInclusiveTimeRange(time.Unix(0, 0), time.Unix(0, 2000)) + return &tr + }(), + want: []index.DocumentResult{ + { + EntityValues: []byte("test2"), + Timestamp: int64(101), + SortedValue: numeric.MustNewPrefixCodedInt64(101, 0), + }, + { + EntityValues: []byte("test3"), + Timestamp: int64(1001), + SortedValue: numeric.MustNewPrefixCodedInt64(1001, 0), + }, + }, + }, + { + name: "Sort by timestamp ascending with time range 500 to 1500", + orderBy: orderBy, + timeRange: func() *timestamp.TimeRange { + tr := timestamp.NewInclusiveTimeRange(time.Unix(0, 500), time.Unix(0, 1500)) + return &tr + }(), + want: []index.DocumentResult{ + { + EntityValues: []byte("test3"), + Timestamp: int64(1001), + SortedValue: numeric.MustNewPrefixCodedInt64(1001, 0), + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + query, err := s.BuildQuery([]index.SeriesMatcher{ + { + Type: index.SeriesMatcherTypePrefix, + Match: []byte("test"), + }, + }, nil, tt.timeRange) + require.NoError(t, err) + iter, err := s.SeriesSort(context.Background(), query, tt.orderBy, 10, tt.fieldKeys) + require.NoError(t, err) + defer iter.Close() + + var got []index.DocumentResult + for iter.Next() { + var g index.DocumentResult + val := iter.Val() + g.DocID = val.DocID + g.EntityValues = val.EntityValues + if len(val.Values) > 0 { + g.Values = maps.Clone(val.Values) + } + g.SortedValue = val.SortedValue + g.Timestamp = val.Timestamp + got = append(got, g) + } + assert.Equal(t, tt.want, got) + }) + } +} + func setupData(tester *require.Assertions, s index.SeriesStore) { series1 := index.Document{ - DocID: 1, EntityValues: []byte("test1"), } series2 := index.Document{ - DocID: 2, EntityValues: []byte("test2"), Fields: []index.Field{ { Key: fieldKeyDuration, Term: convert.Int64ToBytes(int64(100)), Store: true, + Index: true, }, { Key: fieldKeyServiceName, Term: []byte("svc2"), Store: true, + Index: true, + }, + { + Key: fieldKeyStartTime, + Term: convert.Int64ToBytes(int64(100)), + Store: true, + Index: true, + }, + { + Key: index.FieldKey{ + TagName: "short_name", + }, + Term: []byte("t2"), + Store: true, + Index: false, }, }, + Timestamp: int64(101), } series3 := index.Document{ - DocID: 3, EntityValues: []byte("test3"), Fields: []index.Field{ { Key: fieldKeyDuration, Term: convert.Int64ToBytes(int64(500)), Store: true, + Index: true, + }, + { + Key: fieldKeyStartTime, + Term: convert.Int64ToBytes(int64(1000)), + Store: true, + Index: true, + }, + { + Key: index.FieldKey{ + TagName: "short_name", + }, + Term: []byte("t3"), + Store: true, + Index: false, }, }, + Timestamp: int64(1001), } + series4 := index.Document{ + EntityValues: []byte("test4"), + Fields: []index.Field{ + { + Key: fieldKeyDuration, + Term: convert.Int64ToBytes(int64(500)), + Store: true, + Index: true, + }, + { + Key: fieldKeyStartTime, + Term: convert.Int64ToBytes(int64(2000)), + Store: true, + Index: true, + }, + }, + Timestamp: int64(2001), + } + tester.NoError(s.SeriesBatch(index.Batch{ + Documents: []index.Document{series1, series2, series4, series3}, + })) tester.NoError(s.SeriesBatch(index.Batch{ - Documents: []index.Document{series1, series2, series3, series3}, + Documents: []index.Document{series3}, })) } diff --git a/pkg/index/inverted/query.go b/pkg/index/inverted/query.go index f44873528..2dc630930 100644 --- a/pkg/index/inverted/query.go +++ b/pkg/index/inverted/query.go @@ -31,6 +31,7 @@ import ( "github.com/apache/skywalking-banyandb/pkg/convert" "github.com/apache/skywalking-banyandb/pkg/index" "github.com/apache/skywalking-banyandb/pkg/query/logical" + "github.com/apache/skywalking-banyandb/pkg/timestamp" ) var ( @@ -466,3 +467,23 @@ func (m *wildcardNode) MarshalJSON() ([]byte, error) { func (m *wildcardNode) String() string { return convert.JSONToString(m) } + +type timeRangeNode struct { + timeRange *timestamp.TimeRange +} + +func newTimeRangeNode(timeRange *timestamp.TimeRange) *timeRangeNode { + return &timeRangeNode{ + timeRange: timeRange, + } +} + +func (t *timeRangeNode) MarshalJSON() ([]byte, error) { + data := make(map[string]interface{}, 1) + data["time_range"] = t.timeRange.String() + return json.Marshal(data) +} + +func (t *timeRangeNode) String() string { + return convert.JSONToString(t) +} diff --git a/pkg/pb/v1/series.go b/pkg/pb/v1/series.go index a0e768276..542343d94 100644 --- a/pkg/pb/v1/series.go +++ b/pkg/pb/v1/series.go @@ -72,6 +72,7 @@ func (s *Series) MarshalWithWildcard() error { // Unmarshal decodes series from internal Buffer. func (s *Series) Unmarshal(src []byte) error { + s.ID = common.SeriesID(convert.Hash(src)) var err error s.Buffer = s.Buffer[:0] if s.Buffer, src, err = unmarshalEntityValue(s.Buffer, src); err != nil { diff --git a/pkg/query/logical/measure/measure_plan_indexscan_local.go b/pkg/query/logical/measure/measure_plan_indexscan_local.go index bf11e1115..4e2745f87 100644 --- a/pkg/query/logical/measure/measure_plan_indexscan_local.go +++ b/pkg/query/logical/measure/measure_plan_indexscan_local.go @@ -135,29 +135,33 @@ func (i *localIndexScan) Sort(order *logical.OrderBy) { } func (i *localIndexScan) Execute(ctx context.Context) (mit executor.MIterator, err error) { - var orderBy *model.OrderBy - orderByType := model.OrderByTypeTime + var orderBy *index.OrderBy + if i.order != nil { - if i.order.Index != nil { - orderByType = model.OrderByTypeIndex - } - orderBy = &model.OrderBy{ - Index: i.order.Index, + orderBy = &index.OrderBy{ Sort: i.order.Sort, + Index: i.order.Index, + } + if orderBy.Index == nil { + orderBy.Type = index.OrderByTypeTime + } else { + orderBy.Type = index.OrderByTypeIndex } } if i.groupByEntity { - orderByType = model.OrderByTypeSeries + if orderBy == nil { + orderBy = &index.OrderBy{} + } + orderBy.Type = index.OrderByTypeSeries } ec := executor.FromMeasureExecutionContext(ctx) - ctx, stop := i.startSpan(ctx, query.GetTracer(ctx), orderByType, orderBy) + ctx, stop := i.startSpan(ctx, query.GetTracer(ctx), orderBy) defer stop(err) result, err := ec.Query(ctx, model.MeasureQueryOptions{ Name: i.metadata.GetName(), TimeRange: &i.timeRange, Entities: i.entities, Query: i.query, - OrderByType: orderByType, Order: orderBy, TagProjection: i.projectionTags, FieldProjection: i.projectionFields, @@ -269,20 +273,24 @@ func (ei *resultMIterator) Close() error { return ei.err } -func (i *localIndexScan) startSpan(ctx context.Context, tracer *query.Tracer, orderType model.OrderByType, orderBy *model.OrderBy) (context.Context, func(error)) { +func (i *localIndexScan) startSpan(ctx context.Context, tracer *query.Tracer, orderBy *index.OrderBy) (context.Context, func(error)) { if tracer == nil { return ctx, func(error) {} } span, ctx := tracer.StartSpan(ctx, "indexScan-%s", i.metadata) - sortName := modelv1.Sort_name[int32(orderBy.Sort)] - switch orderType { - case model.OrderByTypeTime: - span.Tag("orderBy", "time "+sortName) - case model.OrderByTypeIndex: - span.Tag("orderBy", fmt.Sprintf("indexRule:%s", orderBy.Index.Metadata.Name)) - case model.OrderByTypeSeries: - span.Tag("orderBy", "series") + if orderBy != nil { + sortName := modelv1.Sort_name[int32(orderBy.Sort)] + switch orderBy.Type { + case index.OrderByTypeTime: + span.Tag("orderBy", "time-"+sortName) + case index.OrderByTypeIndex: + span.Tag("orderBy", fmt.Sprintf("indexRule:%s-%s", orderBy.Index.Metadata.Name, sortName)) + case index.OrderByTypeSeries: + span.Tag("orderBy", "series") + } + } else { + span.Tag("orderBy", "time-asc(default)") } span.Tag("details", i.String()) diff --git a/pkg/query/logical/measure/topn_plan_localscan.go b/pkg/query/logical/measure/topn_plan_localscan.go index f068c156c..9608af959 100644 --- a/pkg/query/logical/measure/topn_plan_localscan.go +++ b/pkg/query/logical/measure/topn_plan_localscan.go @@ -27,6 +27,7 @@ import ( commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1" modelv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v1" + "github.com/apache/skywalking-banyandb/pkg/index" "github.com/apache/skywalking-banyandb/pkg/logger" pbv1 "github.com/apache/skywalking-banyandb/pkg/pb/v1" "github.com/apache/skywalking-banyandb/pkg/query/executor" @@ -158,7 +159,7 @@ func (i *localScan) Execute(ctx context.Context) (mit executor.MIterator, err er Name: i.metadata.GetName(), TimeRange: &i.timeRange, Entities: [][]*modelv1.TagValue{i.entity}, - Order: &model.OrderBy{Sort: i.sort}, + Order: &index.OrderBy{Sort: i.sort}, TagProjection: i.projectionTags, FieldProjection: i.projectionFields, }) diff --git a/pkg/query/logical/optimizer.go b/pkg/query/logical/optimizer.go index a087cdbcb..471f78b8c 100644 --- a/pkg/query/logical/optimizer.go +++ b/pkg/query/logical/optimizer.go @@ -74,7 +74,8 @@ func NewPushDownOrder(order *modelv1.QueryOrder) PushDownOrder { // Optimize a Plan by pushing down the query order. func (pdo PushDownOrder) Optimize(plan Plan) (Plan, error) { if v, ok := plan.(Sorter); ok { - if order, err := ParseOrderBy(v.Schema(), pdo.order.GetIndexRuleName(), pdo.order.GetSort()); err == nil { + if order, err := ParseOrderBy(v.Schema(), + pdo.order.GetIndexRuleName(), pdo.order.GetSort()); err == nil && order != nil { v.Sort(order) } else { return nil, err diff --git a/pkg/query/logical/plan.go b/pkg/query/logical/plan.go index 5417b6d81..4d30a1f91 100644 --- a/pkg/query/logical/plan.go +++ b/pkg/query/logical/plan.go @@ -63,6 +63,9 @@ func (o *OrderBy) String() string { // ParseOrderBy parses an OrderBy from a Schema. func ParseOrderBy(s Schema, indexRuleName string, sort modelv1.Sort) (*OrderBy, error) { if indexRuleName == "" { + if sort == modelv1.Sort_SORT_UNSPECIFIED { + return nil, nil + } return &OrderBy{ Sort: sort, }, nil diff --git a/pkg/query/logical/stream/stream_plan_indexscan_local.go b/pkg/query/logical/stream/stream_plan_indexscan_local.go index f7db768a1..eda67111c 100644 --- a/pkg/query/logical/stream/stream_plan_indexscan_local.go +++ b/pkg/query/logical/stream/stream_plan_indexscan_local.go @@ -81,9 +81,9 @@ func (i *localIndexScan) Execute(ctx context.Context) ([]*streamv1.Element, erro if i.result != nil { return BuildElementsFromStreamResult(ctx, i.result), nil } - var orderBy *model.OrderBy + var orderBy *index.OrderBy if i.order != nil { - orderBy = &model.OrderBy{ + orderBy = &index.OrderBy{ Index: i.order.Index, Sort: i.order.Sort, } diff --git a/pkg/query/model/model.go b/pkg/query/model/model.go index 022bcdbc4..768e6fad0 100644 --- a/pkg/query/model/model.go +++ b/pkg/query/model/model.go @@ -22,7 +22,6 @@ import ( "context" "github.com/apache/skywalking-banyandb/api/common" - databasev1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1" modelv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v1" "github.com/apache/skywalking-banyandb/pkg/index" "github.com/apache/skywalking-banyandb/pkg/timestamp" @@ -52,34 +51,15 @@ type TagProjection struct { Names []string } -// OrderBy is the order by rule. -type OrderBy struct { - Index *databasev1.IndexRule - Sort modelv1.Sort -} - -// OrderByType is the type of order by. -type OrderByType int - -const ( - // OrderByTypeTime is the order by time. - OrderByTypeTime OrderByType = iota - // OrderByTypeIndex is the order by index. - OrderByTypeIndex - // OrderByTypeSeries is the order by series. - OrderByTypeSeries -) - // MeasureQueryOptions is the options of a measure query. type MeasureQueryOptions struct { Query index.Query TimeRange *timestamp.TimeRange - Order *OrderBy + Order *index.OrderBy Name string Entities [][]*modelv1.TagValue TagProjection []TagProjection FieldProjection []string - OrderByType OrderByType } // MeasureResult is the result of a query. @@ -104,7 +84,7 @@ type StreamQueryOptions struct { TimeRange *timestamp.TimeRange Entities [][]*modelv1.TagValue Filter index.Filter - Order *OrderBy + Order *index.OrderBy TagProjection []TagProjection MaxElementSize int } diff --git a/pkg/test/measure/testdata/measures/service_traffic.json b/pkg/test/measure/testdata/measures/service_traffic.json index 7edc682cd..887d1adb0 100644 --- a/pkg/test/measure/testdata/measures/service_traffic.json +++ b/pkg/test/measure/testdata/measures/service_traffic.json @@ -39,5 +39,6 @@ "id" ] }, + "index_mode": true, "updated_at": "2021-04-15T01:30:15.01Z" } \ No newline at end of file diff --git a/pkg/timestamp/range.go b/pkg/timestamp/range.go index b0b92afdf..f8db16523 100644 --- a/pkg/timestamp/range.go +++ b/pkg/timestamp/range.go @@ -60,6 +60,22 @@ func (t TimeRange) Overlapping(other TimeRange) bool { return !t.Start.After(other.End) && !other.Start.After(t.End) } +// Include returns whether the TimeRange includes the other TimeRange. +func (t TimeRange) Include(other TimeRange) bool { + var start, end bool + if t.Start.Equal(other.Start) { + start = t.IncludeStart || !other.IncludeStart + } else { + start = !t.Start.After(other.Start) + } + if t.End.Equal(other.End) { + end = t.IncludeEnd || !other.IncludeEnd + } else { + end = !t.End.Before(other.End) + } + return start && end +} + // Duration converts TimeRange to time.Duration. func (t TimeRange) Duration() time.Duration { return t.End.Sub(t.Start) diff --git a/test/cases/measure/data/input/index_mode_all.yaml b/test/cases/measure/data/input/index_mode_all.yaml new file mode 100644 index 000000000..d0c014506 --- /dev/null +++ b/test/cases/measure/data/input/index_mode_all.yaml @@ -0,0 +1,23 @@ +# Licensed to Apache Software Foundation (ASF) under one or more contributor +# license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright +# ownership. Apache Software Foundation (ASF) licenses this file to you 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. + +name: "service_traffic" +groups: [ "sw_metric" ] +tagProjection: + tagFamilies: + - name: "default" + tags: [ "id", "service_id", "name", "short_name", "service_group", "layer" ] diff --git a/test/cases/measure/data/input/index_mode_order_desc.yaml b/test/cases/measure/data/input/index_mode_order_desc.yaml new file mode 100644 index 000000000..0dd0063da --- /dev/null +++ b/test/cases/measure/data/input/index_mode_order_desc.yaml @@ -0,0 +1,26 @@ +# Licensed to Apache Software Foundation (ASF) under one or more contributor +# license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright +# ownership. Apache Software Foundation (ASF) licenses this file to you 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. + +name: "service_traffic" +groups: [ "sw_metric" ] +tagProjection: + tagFamilies: + - name: "default" + tags: [ "id", "service_id", "name", "layer" ] +orderBy: + sort: "SORT_DESC" + indexRuleName: "layer" diff --git a/test/cases/measure/data/input/index_mode_range.yaml b/test/cases/measure/data/input/index_mode_range.yaml new file mode 100644 index 000000000..c881b11cf --- /dev/null +++ b/test/cases/measure/data/input/index_mode_range.yaml @@ -0,0 +1,30 @@ +# Licensed to Apache Software Foundation (ASF) under one or more contributor +# license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright +# ownership. Apache Software Foundation (ASF) licenses this file to you 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. + +name: "service_traffic" +groups: [ "sw_metric" ] +tagProjection: + tagFamilies: + - name: "default" + tags: [ "id", "service_id", "name", "layer" ] +criteria: + condition: + name: "layer" + op: "BINARY_OP_GT" + value: + int: + value: "1" diff --git a/test/cases/measure/data/want/index_mode_all.yaml b/test/cases/measure/data/want/index_mode_all.yaml new file mode 100644 index 000000000..3d3edeb2d --- /dev/null +++ b/test/cases/measure/data/want/index_mode_all.yaml @@ -0,0 +1,107 @@ +# Licensed to Apache Software Foundation (ASF) under one or more contributor +# license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright +# ownership. Apache Software Foundation (ASF) licenses this file to you 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. +dataPoints: +- sid: "15142466043926325685" + tagFamilies: + - name: default + tags: + - key: id + value: + str: + value: "1" + - key: service_id + value: + str: + value: service_1 + - key: name + value: + str: + value: service_name_1 + - key: short_name + value: + str: + value: service_short_name_1 + - key: service_group + value: + str: + value: group1 + - key: layer + value: + int: + value: "1" + timestamp: "2024-11-15T01:02:00Z" + version: "1" +- sid: "3906119849472468294" + tagFamilies: + - name: default + tags: + - key: id + value: + str: + value: "2" + - key: service_id + value: + str: + value: service_2 + - key: name + value: + str: + value: service_name_2 + - key: short_name + value: + str: + value: service_short_name_2 + - key: service_group + value: + str: + value: group1 + - key: layer + value: + int: + value: "2" + timestamp: "2024-11-15T01:03:00Z" + version: "1" +- sid: "12370392692163567533" + tagFamilies: + - name: default + tags: + - key: id + value: + str: + value: "3" + - key: service_id + value: + str: + value: service_3 + - key: name + value: + str: + value: service_name_3 + - key: short_name + value: + str: + value: service_short_name_3 + - key: service_group + value: + str: + value: group1 + - key: layer + value: + int: + value: "1" + timestamp: "2024-11-15T01:04:00Z" + version: "1" diff --git a/test/cases/measure/data/want/index_mode_order_desc.yaml b/test/cases/measure/data/want/index_mode_order_desc.yaml new file mode 100644 index 000000000..470fba1c3 --- /dev/null +++ b/test/cases/measure/data/want/index_mode_order_desc.yaml @@ -0,0 +1,83 @@ +# Licensed to Apache Software Foundation (ASF) under one or more contributor +# license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright +# ownership. Apache Software Foundation (ASF) licenses this file to you 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. +dataPoints: +- sid: "3906119849472468294" + tagFamilies: + - name: default + tags: + - key: id + value: + str: + value: "2" + - key: service_id + value: + str: + value: service_2 + - key: name + value: + str: + value: service_name_2 + - key: layer + value: + int: + value: "2" + timestamp: "2024-11-15T01:40:00Z" + version: "1" +- sid: "15142466043926325685" + tagFamilies: + - name: default + tags: + - key: id + value: + str: + value: "1" + - key: service_id + value: + str: + value: service_1 + - key: name + value: + str: + value: service_name_1 + - key: layer + value: + int: + value: "1" + timestamp: "2024-11-15T01:39:00Z" + version: "1" +- sid: "12370392692163567533" + tagFamilies: + - name: default + tags: + - key: id + value: + str: + value: "3" + - key: service_id + value: + str: + value: service_3 + - key: name + value: + str: + value: service_name_3 + - key: layer + value: + int: + value: "1" + timestamp: "2024-11-15T01:41:00Z" + version: "1" diff --git a/test/cases/measure/data/want/index_mode_range.yaml b/test/cases/measure/data/want/index_mode_range.yaml new file mode 100644 index 000000000..f9c672262 --- /dev/null +++ b/test/cases/measure/data/want/index_mode_range.yaml @@ -0,0 +1,39 @@ +# Licensed to Apache Software Foundation (ASF) under one or more contributor +# license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright +# ownership. Apache Software Foundation (ASF) licenses this file to you 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. +dataPoints: +- sid: "3906119849472468294" + tagFamilies: + - name: default + tags: + - key: id + value: + str: + value: "2" + - key: service_id + value: + str: + value: service_2 + - key: name + value: + str: + value: service_name_2 + - key: layer + value: + int: + value: "2" + timestamp: "2024-11-15T01:43:00Z" + version: "1" diff --git a/test/cases/measure/measure.go b/test/cases/measure/measure.go index 8f843105b..bc31747d6 100644 --- a/test/cases/measure/measure.go +++ b/test/cases/measure/measure.go @@ -72,4 +72,7 @@ var _ = g.DescribeTable("Scanning Measures", verify, g.Entry("all_latency", helpers.Args{Input: "all_latency", Duration: 25 * time.Minute, Offset: -20 * time.Minute}), g.Entry("duplicated in a part", helpers.Args{Input: "duplicated_part", Duration: 25 * time.Minute, Offset: -20 * time.Minute}), g.Entry("match a tag belongs to the entity", helpers.Args{Input: "entity_match", Duration: 25 * time.Minute, Offset: -20 * time.Minute}), + g.Entry("all of index mode", helpers.Args{Input: "index_mode_all", Duration: 25 * time.Minute, Offset: -20 * time.Minute}), + g.Entry("order by desc of index mode", helpers.Args{Input: "index_mode_order_desc", Duration: 25 * time.Minute, Offset: -20 * time.Minute}), + g.Entry("range of index mode", helpers.Args{Input: "index_mode_range", Duration: 25 * time.Minute, Offset: -20 * time.Minute}), ) From b13b5098baa97923fb7763fb540544453175e79f Mon Sep 17 00:00:00 2001 From: Gao Hongtao Date: Fri, 15 Nov 2024 06:28:43 +0000 Subject: [PATCH 5/6] Update metrics stress test Signed-off-by: Gao Hongtao --- test/stress/istio/report.md | 178 +++++----- test/stress/istio/testdata/metrics/data.csv | 307 +++++++++--------- .../istio/testdata/metrics/intermediate.csv | 18 +- test/stress/istio/testdata/metrics/result.csv | 28 +- 4 files changed, 265 insertions(+), 266 deletions(-) diff --git a/test/stress/istio/report.md b/test/stress/istio/report.md index 8089e50f9..28ba80cce 100644 --- a/test/stress/istio/report.md +++ b/test/stress/istio/report.md @@ -16,122 +16,120 @@ VUS: 8192 Machine and OS information: -- CPU: CPU: AMD EPYC 7B12, VendorID: AuthenticAMD, Family: 23, Model: 49, Cores: 8 -- Memory: Total: 47176 MB -- OS: OS: linux, Platform: ubuntu, Platform Family: debian, Platform Version: 21.04 +- CPU: CPU: AMD EPYC 7B13, VendorID: AuthenticAMD, Family: 25, Model: 1, Cores: 16 +- Memory: Total: 15986 MB +- OS: OS: linux, Platform: ubuntu, Platform Family: debian, Platform Version: 24.04 ## Result -written 16186000 items in 38m38.657047557s -throughput: 6980.765015 items/s -throughput(kb/s) 1786.494948 -latency: 2ns +written 16186000 items in 38m25.757182449s +throughput: 7019.819833 items/s +throughput(kb/s) 1796.489734 +latency: 167.193µs -The Memory and Disk usage is improved due to moving indexed value from data files to index files. +CPU and Memory usage are improved due to reducing the updates of the index. ### Problem -Memory: Series Index(bluge) index building took most of the memory. - -CPU: As memory's part, Series Index(bluge) index building took most of the CPU. - -Next step: Optimize the Series Index(bluge) index building. +Series Index Searching took most of the time. Add a cache would be helpful. ## CPU -CPU 95th-percentile: 1.24 cores (8 cores in total) +CPU 95th-percentile: 0.75 cores (16 cores in total) ```bash -Showing top 10 nodes out of 326 +Showing top 10 nodes out of 301 flat flat% sum% cum cum% - 229.15s 7.80% 7.80% 229.15s 7.80% runtime/internal/syscall.Syscall6 - 152.36s 5.18% 12.98% 436.89s 14.86% runtime.mallocgc - 107.72s 3.66% 16.64% 321.02s 10.92% runtime.scanobject - 93.73s 3.19% 19.83% 179.11s 6.09% github.com/blevesearch/vellum.registryCache.entry - 87.88s 2.99% 22.82% 104.15s 3.54% runtime.findObject - 68.76s 2.34% 25.16% 68.96s 2.35% github.com/blevesearch/vellum.(*builderNode).equiv (inline) - 56.29s 1.91% 27.08% 56.29s 1.91% runtime.nextFreeFast (inline) - 49.87s 1.70% 28.77% 49.87s 1.70% runtime.memmove - 46.09s 1.57% 30.34% 69.86s 2.38% runtime.(*mspan).writeHeapBitsSmall - 45.67s 1.55% 31.89% 45.67s 1.55% runtime.memclrNoHeapPointers + 335.22s 18.10% 18.10% 335.22s 18.10% internal/runtime/syscall.Syscall6 + 75.99s 4.10% 22.20% 225.86s 12.19% runtime.mallocgc + 43.86s 2.37% 24.57% 48.40s 2.61% github.com/blevesearch/vellum.(*fstStateV1).atSingle + 39.80s 2.15% 26.72% 39.80s 2.15% runtime.memmove + 39.45s 2.13% 28.85% 39.45s 2.13% runtime.usleep + 36.23s 1.96% 30.80% 218.49s 11.80% github.com/blevesearch/vellum.(*FST).get + 32.43s 1.75% 32.56% 32.43s 1.75% runtime.nextFreeFast (inline) + 30.64s 1.65% 34.21% 30.64s 1.65% runtime.futex + 28.75s 1.55% 35.76% 148.49s 8.02% github.com/blevesearch/vellum.(*decoderV1).stateAt + 27.72s 1.50% 37.26% 27.72s 1.50% runtime.memclrNoHeapPointers ``` ## Heap Profile -`alloc_bytes` 95th-percentile: 947.97 MB. -`heap_inuse_bytes` 95th-percentile: 1.10 GB. -`sys_bytes` 95th-percentile: 1.75 GB. -`stack_inuse_bytes` 95th-percentile: 20.58 MB. +`alloc_bytes` 95th-percentile: 431.5 MB. +`heap_inuse_bytes` 95th-percentile: 452.97 GB. +`sys_bytes` 95th-percentile: 892.03 GB. +`stack_inuse_bytes` 95th-percentile: 2.35 MB. ```bash -Showing top 10 nodes out of 225 +Showing top 10 nodes out of 238 flat flat% sum% cum cum% - 32.36GB 9.63% 9.63% 32.36GB 9.63% github.com/blevesearch/vellum.(*unfinishedNodes).get - 26.01GB 7.74% 17.36% 26.01GB 7.74% reflect.New - 16.32GB 4.86% 22.22% 16.32GB 4.86% github.com/blevesearch/vellum.(*builderNodePool).Get - 13.81GB 4.11% 26.33% 13.81GB 4.11% github.com/apache/skywalking-banyandb/banyand/measure.(*blockPointer).append - 13.74GB 4.09% 30.42% 13.74GB 4.09% github.com/blugelabs/bluge/analysis.TokenFrequency - 12.63GB 3.76% 34.17% 12.63GB 3.76% github.com/blugelabs/ice.(*interim).processDocument.func1.1 - 10.44GB 3.11% 37.28% 10.44GB 3.11% github.com/blevesearch/vellum.(*builderNodeUnfinished).lastCompiled - 9.42GB 2.80% 40.08% 9.42GB 2.80% bytes.growSlice - 8.57GB 2.55% 42.63% 22.18GB 6.60% github.com/apache/skywalking-banyandb/banyand/measure.(*writeCallback).handle - 8.40GB 2.50% 45.13% 8.40GB 2.50% github.com/RoaringBitmap/roaring.(*Bitmap).Iterator + 27.77GB 12.09% 12.09% 27.77GB 12.09% reflect.New + 22.72GB 9.89% 21.98% 22.72GB 9.89% github.com/blugelabs/bluge.Identifier.Term + 8.50GB 3.70% 25.68% 8.50GB 3.70% github.com/blevesearch/vellum.(*decoderV1).stateAt + 7.54GB 3.28% 28.96% 54.19GB 23.59% github.com/apache/skywalking-banyandb/test/stress/istio.ReadAndWriteFromFile.func4 + 7.53GB 3.28% 32.23% 22.32GB 9.72% github.com/apache/skywalking-banyandb/banyand/measure.(*writeCallback).handle + 6.57GB 2.86% 35.09% 6.57GB 2.86% strings.(*Builder).grow + 5.39GB 2.34% 37.44% 5.69GB 2.48% google.golang.org/protobuf/proto.MarshalOptions.marshal + 5.30GB 2.30% 39.74% 5.30GB 2.30% google.golang.org/protobuf/internal/encoding/json.(*Decoder).parseString + 4.95GB 2.15% 41.90% 4.95GB 2.16% github.com/klauspost/compress/s2.encodeBlockBetter + 4.71GB 2.05% 43.95% 4.71GB 2.05% github.com/apache/skywalking-banyandb/banyand/measure.fastFieldAppend ``` ## Disk Usage ```bash -measure: 368 MB -measure/measure-default: 182 MB -measure/measure-default/seg-20240731: 182 MB -measure/measure-default/seg-20240731/shard-0: 147 MB -measure/measure-default/seg-20240731/shard-0/0000000000001b5c: 75 MB -measure/measure-default/seg-20240731/shard-0/0000000000001c0b: 76 kB -measure/measure-default/seg-20240731/shard-0/0000000000001c6c: 41 kB -measure/measure-default/seg-20240731/shard-0/0000000000001d37: 24 MB -measure/measure-default/seg-20240731/shard-0/0000000000001dae: 80 kB -measure/measure-default/seg-20240731/shard-0/0000000000001edc: 22 MB -measure/measure-default/seg-20240731/shard-0/0000000000001fad: 91 kB -measure/measure-default/seg-20240731/shard-0/0000000000001fb1: 218 kB -measure/measure-default/seg-20240731/shard-0/0000000000002018: 588 kB -measure/measure-default/seg-20240731/shard-0/0000000000002078: 23 MB -measure/measure-default/seg-20240731/shard-0/0000000000002081: 879 kB -measure/measure-default/seg-20240731/sidx: 36 MB -measure/measure-minute: 185 MB -measure/measure-minute/seg-20240731: 185 MB -measure/measure-minute/seg-20240731/shard-0: 80 MB -measure/measure-minute/seg-20240731/shard-0/0000000000000e4d: 52 MB -measure/measure-minute/seg-20240731/shard-0/0000000000000f21: 9.3 MB -measure/measure-minute/seg-20240731/shard-0/0000000000000ff8: 9.4 MB -measure/measure-minute/seg-20240731/shard-0/0000000000001068: 296 kB -measure/measure-minute/seg-20240731/shard-0/00000000000010cd: 9.1 MB -measure/measure-minute/seg-20240731/shard-1: 82 MB -measure/measure-minute/seg-20240731/shard-1/0000000000000de3: 51 MB -measure/measure-minute/seg-20240731/shard-1/0000000000000ead: 9.6 MB -measure/measure-minute/seg-20240731/shard-1/0000000000000f82: 9.1 MB -measure/measure-minute/seg-20240731/shard-1/0000000000000fb0: 2.1 MB -measure/measure-minute/seg-20240731/shard-1/0000000000000fe0: 2.3 MB -measure/measure-minute/seg-20240731/shard-1/0000000000000fe6: 1.4 MB -measure/measure-minute/seg-20240731/shard-1/0000000000000fe9: 162 kB -measure/measure-minute/seg-20240731/shard-1/0000000000000ff1: 383 kB -measure/measure-minute/seg-20240731/shard-1/0000000000001017: 1.8 MB -measure/measure-minute/seg-20240731/shard-1/000000000000101b: 229 kB -measure/measure-minute/seg-20240731/shard-1/0000000000001053: 2.8 MB -measure/measure-minute/seg-20240731/shard-1/0000000000001057: 231 kB -measure/measure-minute/seg-20240731/shard-1/0000000000001058: 74 kB -measure/measure-minute/seg-20240731/shard-1/0000000000001063: 477 kB -measure/measure-minute/seg-20240731/sidx: 24 MB +measure: 562 MB +measure/measure-default: 303 MB +measure/measure-default/seg-20241115: 303 MB +measure/measure-default/seg-20241115/shard-0: 271 MB +measure/measure-default/seg-20241115/shard-0/0000000000000b08: 83 MB +measure/measure-default/seg-20241115/shard-0/00000000000014bb: 80 MB +measure/measure-default/seg-20241115/shard-0/0000000000001e6b: 143 kB +measure/measure-default/seg-20241115/shard-0/0000000000001e6d: 329 kB +measure/measure-default/seg-20241115/shard-0/0000000000001e6e: 78 kB +measure/measure-default/seg-20241115/shard-0/0000000000001f35: 83 MB +measure/measure-default/seg-20241115/shard-0/0000000000001f75: 4.5 MB +measure/measure-default/seg-20241115/shard-0/0000000000001ff7: 9.2 MB +measure/measure-default/seg-20241115/shard-0/0000000000002009: 150 kB +measure/measure-default/seg-20241115/shard-0/0000000000002045: 11 MB +measure/measure-default/seg-20241115/sidx: 32 MB +measure/measure-minute: 259 MB +measure/measure-minute/seg-20241115: 259 MB +measure/measure-minute/seg-20241115/shard-0: 120 MB +measure/measure-minute/seg-20241115/shard-0/00000000000004d4: 33 MB +measure/measure-minute/seg-20241115/shard-0/00000000000009c5: 34 MB +measure/measure-minute/seg-20241115/shard-0/0000000000000bc9: 59 kB +measure/measure-minute/seg-20241115/shard-0/0000000000000f2c: 35 MB +measure/measure-minute/seg-20241115/shard-0/0000000000000f8f: 2.9 MB +measure/measure-minute/seg-20241115/shard-0/0000000000000f96: 60 kB +measure/measure-minute/seg-20241115/shard-0/0000000000000fb8: 5.5 MB +measure/measure-minute/seg-20241115/shard-0/0000000000000ff6: 2.8 MB +measure/measure-minute/seg-20241115/shard-0/0000000000000ff7: 60 kB +measure/measure-minute/seg-20241115/shard-0/000000000000101e: 1.8 MB +measure/measure-minute/seg-20241115/shard-0/0000000000001067: 3.3 MB +measure/measure-minute/seg-20241115/shard-0/000000000000106a: 114 kB +measure/measure-minute/seg-20241115/shard-0/000000000000107f: 1.2 MB +measure/measure-minute/seg-20241115/shard-1: 116 MB +measure/measure-minute/seg-20241115/shard-1/0000000000000515: 35 MB +measure/measure-minute/seg-20241115/shard-1/0000000000000a5c: 36 MB +measure/measure-minute/seg-20241115/shard-1/0000000000000da4: 369 kB +measure/measure-minute/seg-20241115/shard-1/0000000000000e67: 116 kB +measure/measure-minute/seg-20241115/shard-1/0000000000000f34: 34 MB +measure/measure-minute/seg-20241115/shard-1/0000000000000fa1: 264 kB +measure/measure-minute/seg-20241115/shard-1/0000000000000ffc: 116 kB +measure/measure-minute/seg-20241115/shard-1/0000000000000ffd: 59 kB +measure/measure-minute/seg-20241115/shard-1/0000000000001030: 11 MB +measure/measure-minute/seg-20241115/sidx: 23 MB ``` ## Disk IO -| Metric | 95th-percentile per second | -|---------------------|-----------------------------| -| read_count | 0.010000 | -| merged_read_count | 0.000000 | -| write_count | 20.978333 | -| merged_write_count | 25.373333 | -| read_bytes | 40.960000 | -| write_bytes | 2603731.626667 | -| io_time(ms) | 13.360000 | -| weighted_io(ms) | 60.828333 | +| Metric | 95th-percentile per second | +|---------------------|----------------------------| +| read_count | 0.06333 | +| merged_read_count | 0.000000 | +| write_count | 14.873333 | +| merged_write_count | 16.700000 | +| read_bytes | 996.693333 | +| write_bytes | 1689586.346667 | +| io_time(ms) | 7.486667 | +| weighted_io(ms) | 121.133333 | diff --git a/test/stress/istio/testdata/metrics/data.csv b/test/stress/istio/testdata/metrics/data.csv index 3b96410df..0e20b4f0b 100644 --- a/test/stress/istio/testdata/metrics/data.csv +++ b/test/stress/istio/testdata/metrics/data.csv @@ -1,153 +1,154 @@ -305855736.000000,328417280.000000,677992856.000000,5799936.000000,81.030000,314598531072.000000,532622512128.000000,19479256.000000,562851.000000,71599307.000000,123424074.000000,1369619356672.000000,2710871929856.000000,49022512.000000,364109771.000000 -253643696.000000,291848192.000000,678255000.000000,3997696.000000,136.900000,314643181568.000000,532622512128.000000,19479256.000000,562851.000000,71600117.000000,123424948.000000,1369619356672.000000,2710964245504.000000,49023032.000000,364111236.000000 -254710048.000000,293011456.000000,678255000.000000,3997696.000000,136.970000,314643189760.000000,532622512128.000000,19479256.000000,562851.000000,71600755.000000,123425610.000000,1369619356672.000000,2711033795584.000000,49023224.000000,364112532.000000 -255879192.000000,294207488.000000,678255000.000000,3997696.000000,137.040000,314643189760.000000,532622512128.000000,19479256.000000,562851.000000,71600792.000000,123425680.000000,1369619356672.000000,2711034364928.000000,49023252.000000,364112564.000000 -339392248.000000,357883904.000000,678255000.000000,6094848.000000,141.770000,314643197952.000000,532622512128.000000,19479256.000000,562851.000000,71600813.000000,123425691.000000,1369619356672.000000,2711034496000.000000,49023284.000000,364112583.000000 -219319848.000000,277151744.000000,759784872.000000,4325376.000000,211.010000,314665136128.000000,532622512128.000000,19479256.000000,562851.000000,71601860.000000,123426944.000000,1369619356672.000000,2711165113344.000000,49023964.000000,364115001.000000 -255773000.000000,299958272.000000,759784872.000000,5308416.000000,211.250000,314667843584.000000,532622512128.000000,19479256.000000,562851.000000,71601916.000000,123427058.000000,1369619356672.000000,2711168451584.000000,49024016.000000,364115053.000000 -256964024.000000,301228032.000000,759784872.000000,5308416.000000,211.310000,314667851776.000000,532622512128.000000,19479256.000000,562851.000000,71601968.000000,123427124.000000,1369619356672.000000,2711169041408.000000,49024052.000000,364115101.000000 -194574488.000000,270991360.000000,759784872.000000,3866624.000000,215.600000,314667851776.000000,532622512128.000000,19479256.000000,562851.000000,71601992.000000,123427158.000000,1369619356672.000000,2711169278976.000000,49024080.000000,364115124.000000 -374225056.000000,398573568.000000,768963000.000000,4718592.000000,281.670000,314718650368.000000,532622512128.000000,19479256.000000,562851.000000,71602926.000000,123428992.000000,1369619356672.000000,2711264330752.000000,49025020.000000,364116840.000000 -240959760.000000,284499968.000000,768963000.000000,5144576.000000,282.240000,314682363904.000000,532622512128.000000,19479256.000000,562851.000000,71603008.000000,123429121.000000,1369619356672.000000,2711271388160.000000,49025084.000000,364116950.000000 -242098304.000000,285908992.000000,768963000.000000,5144576.000000,282.300000,314682363904.000000,532622512128.000000,19479256.000000,562851.000000,71603267.000000,123429232.000000,1369619356672.000000,2711283655680.000000,49025136.000000,364117202.000000 -408027632.000000,429604864.000000,768963000.000000,5832704.000000,295.810000,314682372096.000000,532622512128.000000,19479256.000000,562851.000000,71603277.000000,123429260.000000,1369619356672.000000,2711283811328.000000,49025160.000000,364117211.000000 -302762976.000000,405700608.000000,841383368.000000,5341184.000000,355.340000,314688856064.000000,532622512128.000000,19479256.000000,562851.000000,71604261.000000,123430238.000000,1369619356672.000000,2711415743488.000000,49025720.000000,364119834.000000 -303820816.000000,406913024.000000,841383368.000000,5341184.000000,355.410000,314688856064.000000,532622512128.000000,19479256.000000,562851.000000,71604267.000000,123430252.000000,1369619356672.000000,2711415825408.000000,49025732.000000,364119838.000000 -304942192.000000,408166400.000000,841383368.000000,5341184.000000,355.480000,314688856064.000000,532622512128.000000,19479256.000000,562851.000000,71604333.000000,123430331.000000,1369619356672.000000,2711416722432.000000,49025756.000000,364119902.000000 -307280992.000000,353132544.000000,841383368.000000,6324224.000000,360.210000,314688860160.000000,532622512128.000000,19479256.000000,562851.000000,71604341.000000,123430336.000000,1369619356672.000000,2711416775680.000000,49025772.000000,364119909.000000 -276951840.000000,377872384.000000,841383368.000000,5439488.000000,425.580000,314734006272.000000,532622512128.000000,19479256.000000,562851.000000,71605006.000000,123431062.000000,1369619356672.000000,2711503111168.000000,49026172.000000,364121458.000000 -305161536.000000,404381696.000000,841383368.000000,5505024.000000,425.750000,314734567424.000000,532622512128.000000,19479256.000000,562851.000000,71605105.000000,123431218.000000,1369619356672.000000,2711506879488.000000,49026252.000000,364121533.000000 -306253904.000000,405716992.000000,841383368.000000,5537792.000000,425.830000,314734567424.000000,532622512128.000000,19479256.000000,562851.000000,71605161.000000,123431276.000000,1369619356672.000000,2711507444736.000000,49026284.000000,364121588.000000 -311480424.000000,369655808.000000,841383368.000000,6684672.000000,433.670000,314734567424.000000,532622512128.000000,19479256.000000,562851.000000,71605180.000000,123431305.000000,1369619356672.000000,2711507641344.000000,49026312.000000,364121607.000000 -355662328.000000,420216832.000000,871464392.000000,6946816.000000,499.790000,314761170944.000000,532622512128.000000,19479256.000000,562851.000000,71606250.000000,123432520.000000,1369619356672.000000,2711644345344.000000,49026968.000000,364124140.000000 -357481496.000000,421634048.000000,871464392.000000,7012352.000000,499.860000,314761285632.000000,532622512128.000000,19479256.000000,562851.000000,71606272.000000,123432572.000000,1369619356672.000000,2711644738560.000000,49026996.000000,364124154.000000 -358622664.000000,422887424.000000,871464392.000000,7012352.000000,499.930000,314761289728.000000,532622512128.000000,19479256.000000,562851.000000,71606322.000000,123432638.000000,1369619356672.000000,2711645307904.000000,49027024.000000,364124203.000000 -435911504.000000,463912960.000000,871464392.000000,7208960.000000,508.960000,314761293824.000000,532622512128.000000,19479256.000000,562851.000000,71606344.000000,123432648.000000,1369619356672.000000,2711645438976.000000,49027056.000000,364124221.000000 -462738048.000000,518053888.000000,889997768.000000,7503872.000000,571.800000,314756165632.000000,532622512128.000000,19479256.000000,562851.000000,71607147.000000,123433655.000000,1369619356672.000000,2711740048384.000000,49027604.000000,364125827.000000 -463871216.000000,519503872.000000,889997768.000000,7503872.000000,571.870000,314756165632.000000,532622512128.000000,19479256.000000,562851.000000,71607151.000000,123433668.000000,1369619356672.000000,2711740118016.000000,49027616.000000,364125830.000000 -465032480.000000,520642560.000000,889997768.000000,7503872.000000,571.940000,314756165632.000000,532622512128.000000,19479256.000000,562851.000000,71607214.000000,123433752.000000,1369619356672.000000,2711740892160.000000,49027640.000000,364125897.000000 -519785024.000000,544251904.000000,889997768.000000,7110656.000000,583.620000,314756169728.000000,532622512128.000000,19479256.000000,562851.000000,71607229.000000,123433763.000000,1369619356672.000000,2711740998656.000000,49027664.000000,364125909.000000 -350615368.000000,428613632.000000,944723416.000000,7962624.000000,646.920000,314782023680.000000,532622512128.000000,19479256.000000,562851.000000,71608441.000000,123435261.000000,1369619356672.000000,2711882605568.000000,49028440.000000,364128509.000000 -351709736.000000,429875200.000000,944723416.000000,7962624.000000,646.990000,314782023680.000000,532622512128.000000,19479256.000000,562851.000000,71608445.000000,123435274.000000,1369619356672.000000,2711882675200.000000,49028452.000000,364128512.000000 -352884328.000000,431095808.000000,944723416.000000,7962624.000000,647.060000,314782023680.000000,532622512128.000000,19479256.000000,562851.000000,71608501.000000,123435357.000000,1369619356672.000000,2711883600896.000000,49028484.000000,364128578.000000 -448501768.000000,489857024.000000,944723416.000000,7012352.000000,659.190000,314782031872.000000,532622512128.000000,19479256.000000,562851.000000,71608517.000000,123435380.000000,1369619356672.000000,2711883760640.000000,49028504.000000,364128593.000000 -458298752.000000,541818880.000000,987783656.000000,8323072.000000,722.270000,314762813440.000000,532622512128.000000,19479256.000000,562851.000000,71609452.000000,123436209.000000,1369619356672.000000,2712031224832.000000,49029068.000000,364131885.000000 -459417648.000000,543014912.000000,987783656.000000,8323072.000000,722.330000,314762817536.000000,532622512128.000000,19479256.000000,562851.000000,71609458.000000,123436224.000000,1369619356672.000000,2712031310848.000000,49029084.000000,364131889.000000 -460575584.000000,544374784.000000,987783656.000000,8323072.000000,722.400000,314762817536.000000,532622512128.000000,19479256.000000,562851.000000,71609512.000000,123436306.000000,1369619356672.000000,2712032007168.000000,49029108.000000,364131941.000000 -578158120.000000,605192192.000000,987783656.000000,8486912.000000,736.330000,314762825728.000000,532622512128.000000,19479256.000000,562851.000000,71609535.000000,123436322.000000,1369619356672.000000,2712032166912.000000,49029140.000000,364131964.000000 -552250168.000000,624828416.000000,1021862376.000000,8224768.000000,799.190000,314747748352.000000,532622512128.000000,19479256.000000,562851.000000,71610854.000000,123437665.000000,1369619356672.000000,2712208659456.000000,49029900.000000,364135964.000000 -553382072.000000,626049024.000000,1021862376.000000,8224768.000000,799.260000,314747748352.000000,532622512128.000000,19479256.000000,562851.000000,71610861.000000,123437687.000000,1369619356672.000000,2712208798720.000000,49029924.000000,364135972.000000 -554508440.000000,627564544.000000,1021862376.000000,8224768.000000,799.330000,314747752448.000000,532622512128.000000,19479256.000000,562851.000000,71610916.000000,123437760.000000,1369619356672.000000,2712209437696.000000,49029960.000000,364136030.000000 -514697272.000000,558940160.000000,1021862376.000000,8912896.000000,812.500000,314747760640.000000,532622512128.000000,19479256.000000,562851.000000,71610933.000000,123437778.000000,1369619356672.000000,2712209581056.000000,49029984.000000,364136045.000000 -389808504.000000,480747520.000000,1022517736.000000,8650752.000000,871.590000,314797207552.000000,532622512128.000000,19479256.000000,562851.000000,71611848.000000,123438964.000000,1369619356672.000000,2712312013824.000000,49030612.000000,364137889.000000 -390936616.000000,481910784.000000,1022517736.000000,8650752.000000,871.660000,314797207552.000000,532622512128.000000,19479256.000000,562851.000000,71611854.000000,123438980.000000,1369619356672.000000,2712312103936.000000,49030628.000000,364137894.000000 -392095584.000000,483172352.000000,1022517736.000000,8650752.000000,871.730000,314797207552.000000,532622512128.000000,19479256.000000,562851.000000,71611908.000000,123439052.000000,1369619356672.000000,2712312816640.000000,49030660.000000,364137955.000000 -546401608.000000,587390976.000000,1022517736.000000,8617984.000000,876.640000,314797211648.000000,532622512128.000000,19479256.000000,562851.000000,71611941.000000,123439082.000000,1369619356672.000000,2712313074688.000000,49030692.000000,364137989.000000 -436601888.000000,549380096.000000,1052732920.000000,9601024.000000,945.160000,314793103360.000000,532622512128.000000,19479256.000000,562851.000000,71612937.000000,123440208.000000,1369619356672.000000,2712437945344.000000,49031300.000000,364140485.000000 -447374704.000000,557989888.000000,1052732920.000000,9601024.000000,945.280000,314793078784.000000,532622512128.000000,19479256.000000,562851.000000,71613011.000000,123440336.000000,1369619356672.000000,2712439690240.000000,49031348.000000,364140534.000000 -447780976.000000,558415872.000000,1052732920.000000,9601024.000000,945.340000,314793082880.000000,532622512128.000000,19479256.000000,562851.000000,71613207.000000,123440431.000000,1369619356672.000000,2712441156608.000000,49031396.000000,364140682.000000 -439056616.000000,532365312.000000,1052732920.000000,9994240.000000,952.170000,314793082880.000000,532622512128.000000,19479256.000000,562851.000000,71613213.000000,123440456.000000,1369619356672.000000,2712441283584.000000,49031408.000000,364140687.000000 -593982144.000000,640008192.000000,1065709048.000000,10125312.000000,1017.430000,314813480960.000000,532622512128.000000,19479256.000000,562851.000000,71614302.000000,123442591.000000,1369619356672.000000,2712549970944.000000,49032372.000000,364142543.000000 -601819248.000000,646660096.000000,1065709048.000000,10125312.000000,1017.540000,314814439424.000000,532622512128.000000,19479256.000000,562851.000000,71614477.000000,123442750.000000,1369619356672.000000,2712552166400.000000,49032432.000000,364142665.000000 -603089008.000000,647929856.000000,1065709048.000000,10125312.000000,1017.610000,314814439424.000000,532622512128.000000,19479256.000000,562851.000000,71614538.000000,123442785.000000,1369619356672.000000,2712563323904.000000,49032476.000000,364142791.000000 -528318200.000000,586022912.000000,1065709048.000000,9338880.000000,1023.740000,314814443520.000000,532622512128.000000,19479256.000000,562851.000000,71614649.000000,123442820.000000,1369619356672.000000,2712564065280.000000,49032508.000000,364142876.000000 -630398168.000000,702529536.000000,1151433224.000000,10452992.000000,1091.190000,314837487616.000000,532622512128.000000,19479256.000000,562851.000000,71615594.000000,123443822.000000,1369619356672.000000,2712689730560.000000,49033100.000000,364145217.000000 -631545448.000000,703791104.000000,1151433224.000000,10452992.000000,1091.260000,314837565440.000000,532622512128.000000,19479256.000000,562851.000000,71615605.000000,123443856.000000,1369619356672.000000,2712689914880.000000,49033116.000000,364145224.000000 -632730024.000000,705069056.000000,1151433224.000000,10452992.000000,1091.330000,314837565440.000000,532622512128.000000,19479256.000000,562851.000000,71615661.000000,123443922.000000,1369619356672.000000,2712690697216.000000,49033140.000000,364145279.000000 -649311792.000000,683393024.000000,1151433224.000000,10878976.000000,1099.050000,314837565440.000000,532622512128.000000,19479256.000000,562851.000000,71615678.000000,123443946.000000,1369619356672.000000,2712690865152.000000,49033164.000000,364145295.000000 -683288800.000000,720412672.000000,1151564296.000000,10911744.000000,1163.530000,314868514816.000000,532622512128.000000,19479256.000000,562851.000000,71616515.000000,123445064.000000,1369619356672.000000,2712786048000.000000,49033760.000000,364146992.000000 -684492936.000000,721551360.000000,1151564296.000000,10977280.000000,1163.600000,314868514816.000000,532622512128.000000,19479256.000000,562851.000000,71616547.000000,123445118.000000,1369619356672.000000,2712786420736.000000,49033780.000000,364147026.000000 -685791256.000000,722837504.000000,1151564296.000000,10977280.000000,1163.670000,314868518912.000000,532622512128.000000,19479256.000000,562851.000000,71616579.000000,123445152.000000,1369619356672.000000,2712786781184.000000,49033808.000000,364147057.000000 -627911368.000000,679247872.000000,1151564296.000000,11206656.000000,1174.060000,314868506624.000000,532622512128.000000,19479256.000000,562851.000000,71616600.000000,123445181.000000,1369619356672.000000,2712786985984.000000,49033832.000000,364147075.000000 -561551184.000000,654450688.000000,1185711640.000000,10715136.000000,1241.060000,314841956352.000000,532622512128.000000,19479256.000000,562851.000000,71617895.000000,123446557.000000,1369619356672.000000,2712960173056.000000,49034612.000000,364150408.000000 -562652016.000000,655663104.000000,1185711640.000000,10715136.000000,1241.150000,314841956352.000000,532622512128.000000,19479256.000000,562851.000000,71617900.000000,123446571.000000,1369619356672.000000,2712960275456.000000,49034628.000000,364150412.000000 -563841024.000000,656867328.000000,1185711640.000000,10715136.000000,1241.220000,314841956352.000000,532622512128.000000,19479256.000000,562851.000000,71617961.000000,123446648.000000,1369619356672.000000,2712960840704.000000,49034652.000000,364150475.000000 -470494544.000000,616341504.000000,1185711640.000000,11468800.000000,1251.550000,314841968640.000000,532622512128.000000,19479256.000000,562851.000000,71617967.000000,123446657.000000,1369619356672.000000,2712960902144.000000,49034664.000000,364150479.000000 -622930280.000000,721223680.000000,1194821144.000000,11796480.000000,1312.850000,314880311296.000000,532622512128.000000,19479256.000000,562851.000000,71618732.000000,123447584.000000,1369619356672.000000,2713053627392.000000,49035220.000000,364152039.000000 -624079088.000000,722518016.000000,1194821144.000000,11796480.000000,1312.930000,314880319488.000000,532622512128.000000,19479256.000000,562851.000000,71618765.000000,123447635.000000,1369619356672.000000,2713054012416.000000,49035240.000000,364152082.000000 -625268816.000000,723640320.000000,1194821144.000000,11796480.000000,1312.990000,314880319488.000000,532622512128.000000,19479256.000000,562851.000000,71618816.000000,123447692.000000,1369619356672.000000,2713054589952.000000,49035276.000000,364152124.000000 -655298304.000000,714186752.000000,1194821144.000000,12025856.000000,1324.660000,314880323584.000000,532622512128.000000,19479256.000000,562851.000000,71618841.000000,123447708.000000,1369619356672.000000,2713054757888.000000,49035312.000000,364152146.000000 -829546272.000000,892215296.000000,1228637720.000000,12124160.000000,1387.030000,314889940992.000000,532622512128.000000,19479256.000000,562851.000000,71619902.000000,123448839.000000,1369619356672.000000,2713191506944.000000,49035916.000000,364154676.000000 -830827504.000000,893411328.000000,1228637720.000000,12124160.000000,1387.100000,314889945088.000000,532622512128.000000,19479256.000000,562851.000000,71619908.000000,123448859.000000,1369619356672.000000,2713191613440.000000,49035932.000000,364154683.000000 -831988896.000000,894803968.000000,1228637720.000000,12124160.000000,1387.170000,314889945088.000000,532622512128.000000,19479256.000000,562851.000000,71619963.000000,123448931.000000,1369619356672.000000,2713192252416.000000,49035956.000000,364154740.000000 -495933176.000000,672210944.000000,1228637720.000000,13303808.000000,1401.240000,314889936896.000000,532622512128.000000,19479256.000000,562851.000000,71619980.000000,123448951.000000,1369619356672.000000,2713192403968.000000,49035984.000000,364154757.000000 -416059432.000000,625598464.000000,1237026328.000000,11468800.000000,1462.930000,314884075520.000000,532622512128.000000,19479256.000000,562851.000000,71621087.000000,123450098.000000,1369619356672.000000,2713346245632.000000,49036588.000000,364158382.000000 -417113920.000000,626696192.000000,1237026328.000000,11468800.000000,1463.000000,314884075520.000000,532622512128.000000,19479256.000000,562851.000000,71621093.000000,123450113.000000,1369619356672.000000,2713346331648.000000,49036604.000000,364158387.000000 -418235696.000000,627818496.000000,1237026328.000000,11468800.000000,1463.070000,314884079616.000000,532622512128.000000,19479256.000000,562851.000000,71621154.000000,123450202.000000,1369619356672.000000,2713347179520.000000,49036640.000000,364158465.000000 -753567512.000000,786923520.000000,1237026328.000000,15040512.000000,1467.980000,314884079616.000000,532622512128.000000,19479256.000000,562851.000000,71621204.000000,123450214.000000,1369619356672.000000,2713347437568.000000,49036660.000000,364158499.000000 -808786128.000000,895328256.000000,1292997160.000000,13107200.000000,1535.790000,314906832896.000000,532622512128.000000,19479256.000000,562851.000000,71622238.000000,123451547.000000,1369619356672.000000,2713467433984.000000,49037288.000000,364160883.000000 -835308816.000000,919363584.000000,1292997160.000000,13107200.000000,1535.970000,314898255872.000000,532622512128.000000,19479256.000000,562851.000000,71622307.000000,123451661.000000,1369619356672.000000,2713470149632.000000,49037332.000000,364160969.000000 -836598072.000000,920616960.000000,1292997160.000000,13107200.000000,1536.040000,314898255872.000000,532622512128.000000,19479256.000000,562851.000000,71622340.000000,123451696.000000,1369619356672.000000,2713470501888.000000,49037364.000000,364161005.000000 -664427496.000000,736362496.000000,1292997160.000000,12255232.000000,1544.230000,314898264064.000000,532622512128.000000,19479256.000000,562851.000000,71622371.000000,123451717.000000,1369619356672.000000,2713470714880.000000,49037388.000000,364161034.000000 -827173408.000000,920330240.000000,1311130664.000000,13402112.000000,1608.980000,314921054208.000000,532622512128.000000,19479256.000000,562851.000000,71623284.000000,123452711.000000,1369619356672.000000,2713581626368.000000,49037864.000000,364162974.000000 -829019408.000000,922042368.000000,1311130664.000000,13402112.000000,1609.050000,314921164800.000000,532622512128.000000,19479256.000000,562851.000000,71623306.000000,123452777.000000,1369619356672.000000,2713582072832.000000,49037892.000000,364163004.000000 -830274224.000000,923328512.000000,1311130664.000000,13402112.000000,1609.130000,314921164800.000000,532622512128.000000,19479256.000000,562851.000000,71623407.000000,123452844.000000,1369619356672.000000,2713582932992.000000,49037924.000000,364163092.000000 -824181752.000000,864722944.000000,1311130664.000000,13762560.000000,1619.200000,314921168896.000000,532622512128.000000,19479256.000000,562851.000000,71623427.000000,123452867.000000,1369619356672.000000,2713583109120.000000,49037948.000000,364163112.000000 -525004136.000000,681893888.000000,1370771512.000000,13271040.000000,1680.790000,314955726848.000000,532622512128.000000,19479256.000000,562851.000000,71624233.000000,123453898.000000,1369619356672.000000,2713675129856.000000,49038528.000000,364164725.000000 -526188576.000000,683008000.000000,1370771512.000000,13271040.000000,1680.860000,314955730944.000000,532622512128.000000,19479256.000000,562851.000000,71624240.000000,123453916.000000,1369619356672.000000,2713675256832.000000,49038548.000000,364164731.000000 -526569144.000000,683294720.000000,1370771512.000000,13271040.000000,1680.930000,314955730944.000000,532622512128.000000,19479256.000000,562851.000000,71624299.000000,123454007.000000,1369619356672.000000,2713676112896.000000,49038576.000000,364164794.000000 -904246112.000000,939720704.000000,1370771512.000000,15302656.000000,1693.600000,314955739136.000000,532622512128.000000,19479256.000000,562851.000000,71624317.000000,123454024.000000,1369619356672.000000,2713676256256.000000,49038604.000000,364164811.000000 -849273232.000000,945029120.000000,1400200264.000000,14221312.000000,1756.160000,314954559488.000000,532622512128.000000,19479256.000000,562851.000000,71625582.000000,123456225.000000,1369619356672.000000,2713818780672.000000,49039656.000000,364167543.000000 -850474912.000000,946388992.000000,1400200264.000000,14221312.000000,1756.210000,314954559488.000000,532622512128.000000,19479256.000000,562851.000000,71625589.000000,123456241.000000,1369619356672.000000,2713818899456.000000,49039676.000000,364167548.000000 -851693352.000000,947666944.000000,1400200264.000000,14221312.000000,1756.280000,314954563584.000000,532622512128.000000,19479256.000000,562851.000000,71625853.000000,123456369.000000,1369619356672.000000,2713831318528.000000,49039736.000000,364167805.000000 -865553416.000000,916873216.000000,1400200264.000000,14581760.000000,1767.460000,314954563584.000000,532622512128.000000,19479256.000000,562851.000000,71625880.000000,123456400.000000,1369619356672.000000,2713831560192.000000,49039764.000000,364167834.000000 -981327984.000000,1092034560.000000,1422744648.000000,14680064.000000,1828.730000,314985230336.000000,532622512128.000000,19479256.000000,562851.000000,71626580.000000,123457162.000000,1369619356672.000000,2713920824320.000000,49040244.000000,364169456.000000 -982411776.000000,1093533696.000000,1422744648.000000,14680064.000000,1828.800000,314985230336.000000,532622512128.000000,19479256.000000,562851.000000,71626616.000000,123457209.000000,1369619356672.000000,2713921184768.000000,49040268.000000,364169492.000000 -982834784.000000,1094000640.000000,1422744648.000000,14680064.000000,1828.870000,314985230336.000000,532622512128.000000,19479256.000000,562851.000000,71626650.000000,123457249.000000,1369619356672.000000,2713921635328.000000,49040296.000000,364169528.000000 -790178184.000000,867696640.000000,1422744648.000000,14843904.000000,1842.390000,314985242624.000000,532622512128.000000,19479256.000000,562851.000000,71626668.000000,123457266.000000,1369619356672.000000,2713921778688.000000,49040316.000000,364169544.000000 -592460408.000000,767918080.000000,1422744648.000000,15073280.000000,1904.320000,315023949824.000000,532622512128.000000,19479256.000000,562851.000000,71627746.000000,123458555.000000,1369619356672.000000,2714050171904.000000,49040988.000000,364171844.000000 -593657432.000000,768999424.000000,1422744648.000000,15073280.000000,1904.380000,315023949824.000000,532622512128.000000,19479256.000000,562851.000000,71627781.000000,123458609.000000,1369619356672.000000,2714050565120.000000,49041008.000000,364171879.000000 -594812696.000000,770293760.000000,1422744648.000000,15073280.000000,1904.460000,315023949824.000000,532622512128.000000,19479256.000000,562851.000000,71627817.000000,123458643.000000,1369619356672.000000,2714050933760.000000,49041044.000000,364171913.000000 -622071528.000000,822345728.000000,1422744648.000000,15269888.000000,1919.250000,315023958016.000000,532622512128.000000,19479256.000000,562851.000000,71627827.000000,123458652.000000,1369619356672.000000,2714051011584.000000,49041072.000000,364171921.000000 -703657512.000000,856440832.000000,1494640728.000000,15532032.000000,1982.500000,314982649856.000000,532622512128.000000,19479256.000000,562851.000000,71628927.000000,123459744.000000,1369619356672.000000,2714217292800.000000,49041712.000000,364177557.000000 -704819704.000000,857808896.000000,1494640728.000000,15532032.000000,1982.560000,314982653952.000000,532622512128.000000,19479256.000000,562851.000000,71628933.000000,123459760.000000,1369619356672.000000,2714217382912.000000,49041732.000000,364177562.000000 -705195952.000000,858095616.000000,1494640728.000000,15532032.000000,1982.630000,314982653952.000000,532622512128.000000,19479256.000000,562851.000000,71628996.000000,123459839.000000,1369619356672.000000,2714218099712.000000,49041764.000000,364177623.000000 -608812656.000000,843268096.000000,1494640728.000000,15663104.000000,1997.590000,314982662144.000000,532622512128.000000,19479256.000000,562851.000000,71629018.000000,123459858.000000,1369619356672.000000,2714218267648.000000,49041792.000000,364177644.000000 -713688152.000000,900595712.000000,1494706264.000000,15302656.000000,2056.340000,315004256256.000000,532622512128.000000,19479256.000000,562851.000000,71630124.000000,123461129.000000,1369619356672.000000,2714344248320.000000,49042408.000000,364180114.000000 -714856248.000000,901750784.000000,1494706264.000000,15302656.000000,2056.400000,315004256256.000000,532622512128.000000,19479256.000000,562851.000000,71630131.000000,123461147.000000,1369619356672.000000,2714344371200.000000,49042432.000000,364180120.000000 -716007032.000000,902938624.000000,1494706264.000000,15302656.000000,2056.490000,315004260352.000000,532622512128.000000,19479256.000000,562851.000000,71630186.000000,123461214.000000,1369619356672.000000,2714344944640.000000,49042464.000000,364180174.000000 -985264080.000000,1043243008.000000,1494706264.000000,17694720.000000,2062.930000,315004264448.000000,532622512128.000000,19479256.000000,562851.000000,71630204.000000,123461236.000000,1369619356672.000000,2714345108480.000000,49042488.000000,364180190.000000 -849464448.000000,984743936.000000,1545892968.000000,15761408.000000,2132.510000,315016851456.000000,532622512128.000000,19479256.000000,562851.000000,71631226.000000,123462101.000000,1369619356672.000000,2714498618368.000000,49043080.000000,364183920.000000 -855159480.000000,987545600.000000,1545892968.000000,15761408.000000,2132.600000,315017502720.000000,532622512128.000000,19479256.000000,562851.000000,71631253.000000,123462161.000000,1369619356672.000000,2714499589120.000000,49043112.000000,364183940.000000 -856285272.000000,988782592.000000,1545892968.000000,15761408.000000,2132.670000,315017502720.000000,532622512128.000000,19479256.000000,562851.000000,71631310.000000,123462222.000000,1369619356672.000000,2714500146176.000000,49043148.000000,364183987.000000 -717144064.000000,882286592.000000,1545892968.000000,16777216.000000,2140.010000,315017515008.000000,532622512128.000000,19479256.000000,562851.000000,71631334.000000,123462249.000000,1369619356672.000000,2714500355072.000000,49043176.000000,364184010.000000 -1127243488.000000,1224171520.000000,1568175208.000000,16875520.000000,2205.560000,315014066176.000000,532622512128.000000,19479256.000000,562851.000000,71632209.000000,123463155.000000,1369619356672.000000,2714614715392.000000,49043716.000000,364186276.000000 -1128451776.000000,1225441280.000000,1568175208.000000,16875520.000000,2205.640000,315014066176.000000,532622512128.000000,19479256.000000,562851.000000,71632214.000000,123463168.000000,1369619356672.000000,2714614813696.000000,49043732.000000,364186280.000000 -1129683248.000000,1226711040.000000,1568175208.000000,16875520.000000,2205.700000,315014066176.000000,532622512128.000000,19479256.000000,562851.000000,71632269.000000,123463231.000000,1369619356672.000000,2714615526400.000000,49043764.000000,364186329.000000 -666545256.000000,882573312.000000,1568175208.000000,17072128.000000,2215.330000,315014070272.000000,532622512128.000000,19479256.000000,562851.000000,71632287.000000,123463251.000000,1369619356672.000000,2714615682048.000000,49043796.000000,364186345.000000 -672478616.000000,864477184.000000,1572369512.000000,16646144.000000,2280.760000,315053543424.000000,532622512128.000000,19479256.000000,562851.000000,71633263.000000,123464443.000000,1369619356672.000000,2714728408064.000000,49044420.000000,364188350.000000 -673620928.000000,865558528.000000,1572369512.000000,16646144.000000,2280.830000,315053543424.000000,532622512128.000000,19479256.000000,562851.000000,71633267.000000,123464456.000000,1369619356672.000000,2714728477696.000000,49044428.000000,364188353.000000 -674780304.000000,866672640.000000,1572369512.000000,16678912.000000,2280.910000,315053543424.000000,532622512128.000000,19479256.000000,562851.000000,71633329.000000,123464521.000000,1369619356672.000000,2714729325568.000000,49044452.000000,364188412.000000 -994481000.000000,1052557312.000000,1572369512.000000,17629184.000000,2291.030000,315053551616.000000,532622512128.000000,19479256.000000,562851.000000,71633347.000000,123464553.000000,1369619356672.000000,2714729530368.000000,49044480.000000,364188432.000000 -812500464.000000,985030656.000000,1623425144.000000,17432576.000000,2354.210000,315063529472.000000,532622512128.000000,19479256.000000,562851.000000,71634377.000000,123465823.000000,1369619356672.000000,2714846696448.000000,49045108.000000,364190796.000000 -813687280.000000,986193920.000000,1623425144.000000,17432576.000000,2354.280000,315063533568.000000,532622512128.000000,19479256.000000,562851.000000,71634381.000000,123465837.000000,1369619356672.000000,2714846770176.000000,49045120.000000,364190799.000000 -813987424.000000,986439680.000000,1623425144.000000,17432576.000000,2354.350000,315063533568.000000,532622512128.000000,19479256.000000,562851.000000,71634486.000000,123465906.000000,1369619356672.000000,2714847650816.000000,49045160.000000,364190893.000000 -711556800.000000,938696704.000000,1623425144.000000,17858560.000000,2367.070000,315063541760.000000,532622512128.000000,19479256.000000,562851.000000,71634507.000000,123465928.000000,1369619356672.000000,2714847826944.000000,49045188.000000,364190911.000000 -638610360.000000,910008320.000000,1691847816.000000,16842752.000000,2427.850000,315078250496.000000,532622512128.000000,19479256.000000,562851.000000,71635556.000000,123467201.000000,1369619356672.000000,2714963448832.000000,49045816.000000,364193043.000000 -639694808.000000,911040512.000000,1691847816.000000,16842752.000000,2427.930000,315078250496.000000,532622512128.000000,19479256.000000,562851.000000,71635561.000000,123467214.000000,1369619356672.000000,2714963547136.000000,49045832.000000,364193047.000000 -640850744.000000,912146432.000000,1691847816.000000,16842752.000000,2428.000000,315078254592.000000,532622512128.000000,19479256.000000,562851.000000,71635624.000000,123467286.000000,1369619356672.000000,2714964141056.000000,49045856.000000,364193101.000000 -806852704.000000,982786048.000000,1691847816.000000,18153472.000000,2442.590000,315078262784.000000,532622512128.000000,19479256.000000,562851.000000,71635643.000000,123467300.000000,1369619356672.000000,2714964276224.000000,49045876.000000,364193116.000000 -739564168.000000,915890176.000000,1693027464.000000,18284544.000000,2503.540000,315081498624.000000,532622512128.000000,19479256.000000,562851.000000,71636821.000000,123469266.000000,1369619356672.000000,2715116307456.000000,49046880.000000,364197309.000000 -740773384.000000,917241856.000000,1693027464.000000,18284544.000000,2503.600000,315081498624.000000,532622512128.000000,19479256.000000,562851.000000,71636827.000000,123469283.000000,1369619356672.000000,2715116401664.000000,49046892.000000,364197315.000000 -741962552.000000,918339584.000000,1693027464.000000,18284544.000000,2503.670000,315081498624.000000,532622512128.000000,19479256.000000,562851.000000,71637027.000000,123469399.000000,1369619356672.000000,2715128566784.000000,49046940.000000,364197549.000000 -699075344.000000,952786944.000000,1693027464.000000,18710528.000000,2516.390000,315081502720.000000,532622512128.000000,19479256.000000,562851.000000,71637115.000000,123469433.000000,1369619356672.000000,2715129111552.000000,49046972.000000,364197616.000000 -860078672.000000,1024188416.000000,1701481608.000000,18808832.000000,2577.450000,315101392896.000000,532622512128.000000,19479256.000000,562851.000000,71638125.000000,123470594.000000,1369619356672.000000,2715243373568.000000,49047604.000000,364199714.000000 -861296912.000000,1025490944.000000,1701481608.000000,18808832.000000,2577.520000,315101392896.000000,532622512128.000000,19479256.000000,562851.000000,71638129.000000,123470607.000000,1369619356672.000000,2715243443200.000000,49047612.000000,364199718.000000 -862507872.000000,1026842624.000000,1701481608.000000,18808832.000000,2577.600000,315101392896.000000,532622512128.000000,19479256.000000,562851.000000,71638183.000000,123470656.000000,1369619356672.000000,2715243918336.000000,49047640.000000,364199764.000000 -943308120.000000,1065197568.000000,1701481608.000000,19136512.000000,2592.130000,315101392896.000000,532622512128.000000,19479256.000000,562851.000000,71638195.000000,123470664.000000,1369619356672.000000,2715244000256.000000,49047664.000000,364199774.000000 -763948960.000000,1050279936.000000,1743755416.000000,17235968.000000,2657.570000,314927493120.000000,532622512128.000000,19479256.000000,562851.000000,71639484.000000,123471703.000000,1369619356672.000000,2715438994432.000000,49048344.000000,364208889.000000 -765004256.000000,1051582464.000000,1743755416.000000,17301504.000000,2657.630000,314927497216.000000,532622512128.000000,19479256.000000,562851.000000,71639490.000000,123471751.000000,1369619356672.000000,2715439215616.000000,49048372.000000,364208912.000000 -766091024.000000,1052852224.000000,1743755416.000000,17301504.000000,2657.700000,314927497216.000000,532622512128.000000,19479256.000000,562851.000000,71639534.000000,123471825.000000,1369619356672.000000,2715439768576.000000,49048400.000000,364208954.000000 -1117176560.000000,1169891328.000000,1743755416.000000,19431424.000000,2663.720000,314927501312.000000,532622512128.000000,19479256.000000,562851.000000,71639559.000000,123471847.000000,1369619356672.000000,2715439961088.000000,49048428.000000,364208977.000000 -978271928.000000,1221427200.000000,1816437928.000000,18350080.000000,2744.140000,314823303168.000000,532622512128.000000,19479256.000000,562851.000000,71640951.000000,123473013.000000,1369619356672.000000,2715655197696.000000,49049172.000000,364214936.000000 -1025150440.000000,1239318528.000000,1816437928.000000,19464192.000000,2744.430000,314826928128.000000,532622512128.000000,19479256.000000,562851.000000,71641000.000000,123473094.000000,1369619356672.000000,2715659293696.000000,49049220.000000,364214996.000000 -1026299384.000000,1240563712.000000,1816437928.000000,19464192.000000,2744.500000,314826932224.000000,532622512128.000000,19479256.000000,562851.000000,71641061.000000,123473169.000000,1369619356672.000000,2715659932672.000000,49049252.000000,364215050.000000 -1219366472.000000,1269211136.000000,1816437928.000000,20021248.000000,2759.620000,314826940416.000000,532622512128.000000,19479256.000000,562851.000000,71641086.000000,123473200.000000,1369619356672.000000,2715660162048.000000,49049272.000000,364215071.000000 -751104224.000000,987734016.000000,1817289896.000000,19988480.000000,2818.570000,314832457728.000000,532622512128.000000,19479256.000000,562851.000000,71642086.000000,123474389.000000,1369619356672.000000,2715779834880.000000,49049904.000000,364217289.000000 -752259784.000000,988815360.000000,1817289896.000000,19988480.000000,2818.620000,314832457728.000000,532622512128.000000,19479256.000000,562851.000000,71642092.000000,123474415.000000,1369619356672.000000,2715779965952.000000,49049924.000000,364217300.000000 -753416904.000000,989954048.000000,1817289896.000000,19988480.000000,2818.710000,314832457728.000000,532622512128.000000,19479256.000000,562851.000000,71642150.000000,123474479.000000,1369619356672.000000,2715780551680.000000,49049956.000000,364217349.000000 -1120425984.000000,1199390720.000000,1817289896.000000,21168128.000000,2825.740000,314832461824.000000,532622512128.000000,19479256.000000,562851.000000,71642171.000000,123474490.000000,1369619356672.000000,2715780682752.000000,49049988.000000,364217368.000000 -899177200.000000,1112252416.000000,1834329256.000000,20447232.000000,2892.840000,314850746368.000000,532622512128.000000,19479256.000000,562851.000000,71643039.000000,123475499.000000,1369619356672.000000,2715888276480.000000,49050516.000000,364219231.000000 -901678232.000000,1114161152.000000,1834329256.000000,20447232.000000,2892.930000,314850906112.000000,532622512128.000000,19479256.000000,562851.000000,71643061.000000,123475545.000000,1369619356672.000000,2715888690176.000000,49050544.000000,364219244.000000 -901985880.000000,1114718208.000000,1834329256.000000,20447232.000000,2892.990000,314850906112.000000,532622512128.000000,19479256.000000,562851.000000,71643117.000000,123475607.000000,1369619356672.000000,2715889304576.000000,49050580.000000,364219298.000000 -848858120.000000,1071980544.000000,1834329256.000000,21004288.000000,2902.110000,314850910208.000000,532622512128.000000,19479256.000000,562851.000000,71643139.000000,123475633.000000,1369619356672.000000,2715889501184.000000,49050604.000000,364219319.000000 +304956928.000000,352313344.000000,891505096.000000,2195456.000000,130.380000,63312080896.000000,132011507712.000000,114108.000000,29935.000000,896750.000000,922431.000000,3107416576.000000,88043596800.000000,448943.000000,6558766.000000 +231226472.000000,262070272.000000,891505096.000000,2195456.000000,182.710000,63356002304.000000,132011507712.000000,114276.000000,29935.000000,898547.000000,923917.000000,3129371136.000000,88160779264.000000,449522.000000,6562686.000000 +408990328.000000,434716672.000000,891505096.000000,2228224.000000,190.070000,63368527872.000000,132011507712.000000,114276.000000,29935.000000,898671.000000,924101.000000,3129371136.000000,88174476288.000000,449576.000000,6562871.000000 +410505752.000000,436174848.000000,891505096.000000,2228224.000000,190.120000,63368531968.000000,132011507712.000000,114283.000000,29935.000000,898784.000000,924154.000000,3129412096.000000,88175356928.000000,449586.000000,6562951.000000 +412024072.000000,437755904.000000,891505096.000000,2228224.000000,190.170000,63368540160.000000,132011507712.000000,114283.000000,29935.000000,898822.000000,924276.000000,3129412096.000000,88176045056.000000,449600.000000,6562997.000000 +208037424.000000,243539968.000000,891505096.000000,2195456.000000,223.080000,63383695360.000000,132011507712.000000,114284.000000,29935.000000,900046.000000,925651.000000,3129424384.000000,88319441920.000000,450208.000000,6570442.000000 +409157816.000000,437305344.000000,891505096.000000,2162688.000000,234.630000,63381020672.000000,132011507712.000000,114284.000000,29935.000000,900424.000000,926004.000000,3129424384.000000,88371100672.000000,450321.000000,6571231.000000 +410423256.000000,438910976.000000,891505096.000000,2162688.000000,234.690000,63381037056.000000,132011507712.000000,114284.000000,29935.000000,900431.000000,926061.000000,3129424384.000000,88371371008.000000,450334.000000,6571249.000000 +410973128.000000,439296000.000000,891505096.000000,2162688.000000,234.740000,63381041152.000000,132011507712.000000,114284.000000,29935.000000,900493.000000,926108.000000,3129424384.000000,88371874816.000000,450338.000000,6571305.000000 +411175896.000000,430006272.000000,891505096.000000,2326528.000000,274.890000,63392956416.000000,132011507712.000000,114284.000000,29935.000000,900875.000000,926464.000000,3129424384.000000,88394046464.000000,450450.000000,6571716.000000 +418392872.000000,440909824.000000,891505096.000000,2326528.000000,277.490000,63400587264.000000,132011507712.000000,114284.000000,29935.000000,900988.000000,926648.000000,3129424384.000000,88406891520.000000,450505.000000,6571892.000000 +419773416.000000,442236928.000000,891505096.000000,2326528.000000,277.550000,63400587264.000000,132011507712.000000,114284.000000,29935.000000,901129.000000,926681.000000,3129424384.000000,88407702528.000000,450515.000000,6571993.000000 +421152928.000000,443867136.000000,891505096.000000,2326528.000000,277.600000,63400591360.000000,132011507712.000000,114284.000000,29935.000000,901166.000000,926827.000000,3129424384.000000,88408452096.000000,450521.000000,6572033.000000 +223244104.000000,268509184.000000,891505096.000000,2260992.000000,314.080000,63415857152.000000,132011507712.000000,114291.000000,29935.000000,901476.000000,927285.000000,3129485824.000000,88434215936.000000,450662.000000,6572421.000000 +386948072.000000,416612352.000000,891505096.000000,2260992.000000,320.310000,63427706880.000000,132011507712.000000,114291.000000,29935.000000,901590.000000,927448.000000,3129485824.000000,88446999552.000000,450712.000000,6572589.000000 +388280160.000000,418308096.000000,891505096.000000,2260992.000000,320.360000,63427710976.000000,132011507712.000000,114291.000000,29935.000000,901690.000000,927478.000000,3129485824.000000,88447593472.000000,450718.000000,6572661.000000 +388893768.000000,418848768.000000,891505096.000000,2260992.000000,320.420000,63427715072.000000,132011507712.000000,114291.000000,29935.000000,901717.000000,927508.000000,3129485824.000000,88447826944.000000,450721.000000,6572689.000000 +276385360.000000,298213376.000000,891505096.000000,2293760.000000,356.210000,63455547392.000000,132011507712.000000,114291.000000,29935.000000,902104.000000,928007.000000,3129485824.000000,88491613184.000000,450887.000000,6573339.000000 +397554384.000000,416915456.000000,891505096.000000,2293760.000000,367.020000,63435976704.000000,132011507712.000000,114291.000000,29935.000000,902418.000000,928306.000000,3129485824.000000,88533154816.000000,451006.000000,6573995.000000 +398950392.000000,418791424.000000,891505096.000000,2293760.000000,367.070000,63435980800.000000,132011507712.000000,114291.000000,29935.000000,902495.000000,928344.000000,3129485824.000000,88533646336.000000,451011.000000,6574052.000000 +400392888.000000,420315136.000000,891505096.000000,2326528.000000,367.130000,63435984896.000000,132011507712.000000,114291.000000,29935.000000,902516.000000,928377.000000,3129485824.000000,88533867520.000000,451014.000000,6574071.000000 +392205808.000000,409747456.000000,891767240.000000,2359296.000000,406.220000,63447695360.000000,132011507712.000000,114291.000000,29935.000000,902862.000000,928928.000000,3129485824.000000,88557689856.000000,451155.000000,6574428.000000 +372634992.000000,398794752.000000,891767240.000000,2326528.000000,410.020000,63455338496.000000,132011507712.000000,114291.000000,29935.000000,902974.000000,929099.000000,3129485824.000000,88570514432.000000,451205.000000,6574606.000000 +374031912.000000,400465920.000000,891767240.000000,2326528.000000,410.070000,63455342592.000000,132011507712.000000,114291.000000,29935.000000,903016.000000,929128.000000,3129485824.000000,88570813440.000000,451206.000000,6574639.000000 +374705960.000000,401121280.000000,891767240.000000,2326528.000000,410.120000,63455346688.000000,132011507712.000000,114291.000000,29935.000000,903045.000000,929165.000000,3129485824.000000,88571083776.000000,451212.000000,6574671.000000 +271383544.000000,292659200.000000,891767240.000000,2293760.000000,445.490000,63471034368.000000,132011507712.000000,114291.000000,29935.000000,903330.000000,929633.000000,3129485824.000000,88594013184.000000,451340.000000,6575020.000000 +389958152.000000,414695424.000000,891767240.000000,2228224.000000,454.510000,63478394880.000000,132011507712.000000,114291.000000,29935.000000,903623.000000,929962.000000,3129485824.000000,88628444160.000000,451460.000000,6575522.000000 +391336840.000000,416325632.000000,891767240.000000,2228224.000000,454.570000,63478398976.000000,132011507712.000000,114291.000000,29935.000000,903720.000000,930021.000000,3129485824.000000,88629222400.000000,451469.000000,6575593.000000 +391931928.000000,417103872.000000,891767240.000000,2228224.000000,454.620000,63478403072.000000,132011507712.000000,114291.000000,29935.000000,903724.000000,930022.000000,3129485824.000000,88629242880.000000,451469.000000,6575596.000000 +250797888.000000,273629184.000000,891767240.000000,2359296.000000,487.270000,63509241856.000000,132011507712.000000,114291.000000,29935.000000,903913.000000,930320.000000,3129485824.000000,88649321472.000000,451549.000000,6575903.000000 +379917584.000000,404873216.000000,891767240.000000,2359296.000000,498.870000,63491731456.000000,132011507712.000000,114291.000000,29935.000000,904228.000000,930623.000000,3129485824.000000,88690715648.000000,451678.000000,6576522.000000 +381337416.000000,406380544.000000,891767240.000000,2359296.000000,498.920000,63491739648.000000,132011507712.000000,114291.000000,29935.000000,904264.000000,930651.000000,3129485824.000000,88691043328.000000,451680.000000,6576552.000000 +382674072.000000,407912448.000000,891767240.000000,2359296.000000,498.980000,63491743744.000000,132011507712.000000,114291.000000,29935.000000,904286.000000,930685.000000,3129485824.000000,88691272704.000000,451684.000000,6576572.000000 +387419728.000000,403865600.000000,891767240.000000,2359296.000000,537.410000,63398203392.000000,132011507712.000000,114300.000000,29935.000000,904592.000000,931135.000000,3129637376.000000,88719166464.000000,451847.000000,6577006.000000 +310743880.000000,339705856.000000,891767240.000000,2260992.000000,542.400000,63510138880.000000,132011507712.000000,114309.000000,29935.000000,905106.000000,931358.000000,3129780736.000000,88836533248.000000,452363.000000,6597231.000000 +312096792.000000,341188608.000000,891767240.000000,2260992.000000,542.450000,63510147072.000000,132011507712.000000,114309.000000,29935.000000,905298.000000,931398.000000,3129780736.000000,88837696512.000000,452375.000000,6597375.000000 +312572008.000000,341786624.000000,891767240.000000,2260992.000000,542.510000,63510151168.000000,132011507712.000000,114309.000000,29935.000000,905343.000000,931453.000000,3129780736.000000,88838106112.000000,452379.000000,6597425.000000 +370008352.000000,385966080.000000,891767240.000000,2392064.000000,576.670000,63526109184.000000,132011507712.000000,114309.000000,29935.000000,905613.000000,931892.000000,3129780736.000000,88860044288.000000,452477.000000,6597764.000000 +297477640.000000,322281472.000000,891767240.000000,2260992.000000,585.800000,63531696128.000000,132011507712.000000,114310.000000,29935.000000,905909.000000,932236.000000,3129784832.000000,88896052224.000000,452592.000000,6598299.000000 +298858936.000000,323870720.000000,891767240.000000,2293760.000000,585.850000,63531700224.000000,132011507712.000000,114310.000000,29935.000000,905947.000000,932275.000000,3129784832.000000,88896547840.000000,452595.000000,6598339.000000 +300201784.000000,325386240.000000,891767240.000000,2326528.000000,585.910000,63531704320.000000,132011507712.000000,114310.000000,29935.000000,905971.000000,932307.000000,3129784832.000000,88896777216.000000,452598.000000,6598362.000000 +228412872.000000,285589504.000000,892029384.000000,2293760.000000,632.720000,63522222080.000000,132011507712.000000,114310.000000,29935.000000,906664.000000,933009.000000,3129784832.000000,88988400640.000000,452877.000000,6600300.000000 +326778144.000000,356925440.000000,892029384.000000,2293760.000000,633.280000,63529590784.000000,132011507712.000000,114310.000000,29935.000000,906760.000000,933177.000000,3129784832.000000,88996719616.000000,452926.000000,6600423.000000 +328214448.000000,358277120.000000,892029384.000000,2293760.000000,633.330000,63529594880.000000,132011507712.000000,114310.000000,29935.000000,906805.000000,933240.000000,3129784832.000000,88997342208.000000,452929.000000,6600463.000000 +328814928.000000,358785024.000000,892029384.000000,2293760.000000,633.390000,63529594880.000000,132011507712.000000,114310.000000,29935.000000,906824.000000,933278.000000,3129784832.000000,88997575680.000000,452932.000000,6600479.000000 +306967784.000000,325386240.000000,892029384.000000,2359296.000000,674.080000,63526989824.000000,132011507712.000000,114310.000000,29935.000000,908096.000000,934851.000000,3129784832.000000,89170488320.000000,453610.000000,6612154.000000 +256065960.000000,293167104.000000,892029384.000000,2293760.000000,678.980000,63532638208.000000,132011507712.000000,114310.000000,29935.000000,908278.000000,935027.000000,3129784832.000000,89181756416.000000,453667.000000,6612351.000000 +257365096.000000,294584320.000000,892029384.000000,2293760.000000,679.030000,63532642304.000000,132011507712.000000,114310.000000,29935.000000,908343.000000,935087.000000,3129784832.000000,89182452736.000000,453671.000000,6612397.000000 +257913448.000000,295165952.000000,892029384.000000,2293760.000000,679.090000,63532650496.000000,132011507712.000000,114310.000000,29935.000000,908364.000000,935130.000000,3129784832.000000,89182714880.000000,453674.000000,6612416.000000 +244023184.000000,289947648.000000,892029384.000000,2326528.000000,715.110000,63584702464.000000,132011507712.000000,114310.000000,29935.000000,908541.000000,935402.000000,3129784832.000000,89202355200.000000,453744.000000,6612689.000000 +343033568.000000,367419392.000000,892029384.000000,2260992.000000,726.600000,63512010752.000000,132011507712.000000,114310.000000,29935.000000,909039.000000,935662.000000,3129784832.000000,89299131392.000000,454128.000000,6618516.000000 +344389104.000000,368910336.000000,892029384.000000,2260992.000000,726.660000,63512018944.000000,132011507712.000000,114310.000000,29935.000000,909082.000000,935694.000000,3129784832.000000,89299504128.000000,454131.000000,6618553.000000 +344983088.000000,369524736.000000,892029384.000000,2260992.000000,726.710000,63512018944.000000,132011507712.000000,114310.000000,29935.000000,909104.000000,935727.000000,3129784832.000000,89299729408.000000,454137.000000,6618578.000000 +425435408.000000,441245696.000000,892029384.000000,2359296.000000,768.170000,63521304576.000000,132011507712.000000,114319.000000,29935.000000,909431.000000,936189.000000,3129858560.000000,89333423104.000000,454285.000000,6619057.000000 +211751128.000000,242704384.000000,892029384.000000,2260992.000000,770.830000,63526207488.000000,132011507712.000000,114319.000000,29935.000000,909653.000000,936436.000000,3129858560.000000,89359457280.000000,454369.000000,6619413.000000 +213035304.000000,244072448.000000,892029384.000000,2260992.000000,770.890000,63526207488.000000,132011507712.000000,114319.000000,29935.000000,909739.000000,936473.000000,3129858560.000000,89360108544.000000,454373.000000,6619482.000000 +214295912.000000,245637120.000000,892029384.000000,2293760.000000,770.940000,63526211584.000000,132011507712.000000,114319.000000,29935.000000,909760.000000,936507.000000,3129858560.000000,89360333824.000000,454377.000000,6619503.000000 +315074768.000000,334110720.000000,892029384.000000,2359296.000000,808.180000,63534690304.000000,132011507712.000000,114319.000000,29935.000000,910200.000000,937127.000000,3129858560.000000,89396222976.000000,454571.000000,6620082.000000 +284700328.000000,319160320.000000,892029384.000000,2260992.000000,814.740000,63542231040.000000,132011507712.000000,114319.000000,29935.000000,910312.000000,937295.000000,3129858560.000000,89408973824.000000,454627.000000,6620240.000000 +285964976.000000,320569344.000000,892029384.000000,2260992.000000,814.790000,63542235136.000000,132011507712.000000,114319.000000,29935.000000,910344.000000,937333.000000,3129858560.000000,89409272832.000000,454629.000000,6620263.000000 +286545616.000000,321077248.000000,892029384.000000,2260992.000000,814.850000,63542235136.000000,132011507712.000000,114319.000000,29935.000000,910363.000000,937364.000000,3129858560.000000,89409477632.000000,454633.000000,6620284.000000 +376810880.000000,393887744.000000,892029384.000000,2359296.000000,848.190000,63558877184.000000,132011507712.000000,114319.000000,29935.000000,910643.000000,937784.000000,3129858560.000000,89432144896.000000,454761.000000,6620618.000000 +307697808.000000,334110720.000000,892029384.000000,2260992.000000,856.830000,63567491072.000000,132011507712.000000,114319.000000,29935.000000,910850.000000,938032.000000,3129858560.000000,89455156224.000000,454829.000000,6620940.000000 +309038000.000000,335486976.000000,892029384.000000,2260992.000000,856.880000,63567491072.000000,132011507712.000000,114319.000000,29935.000000,910850.000000,938032.000000,3129858560.000000,89455156224.000000,454829.000000,6620940.000000 +310551488.000000,337117184.000000,892029384.000000,2293760.000000,856.930000,63567491072.000000,132011507712.000000,114319.000000,29935.000000,910927.000000,938097.000000,3129858560.000000,89455823872.000000,454839.000000,6621008.000000 +434090184.000000,450969600.000000,892029384.000000,2293760.000000,899.740000,63568433152.000000,132011507712.000000,114319.000000,29935.000000,911402.000000,938642.000000,3129858560.000000,89511775232.000000,455043.000000,6621780.000000 +319961176.000000,349306880.000000,892029384.000000,2195456.000000,900.400000,63576969216.000000,132011507712.000000,114319.000000,29935.000000,911505.000000,938803.000000,3129858560.000000,89521241088.000000,455092.000000,6621914.000000 +321353192.000000,350920704.000000,892029384.000000,2195456.000000,900.460000,63576969216.000000,132011507712.000000,114319.000000,29935.000000,911589.000000,938841.000000,3129858560.000000,89521802240.000000,455098.000000,6621981.000000 +322672152.000000,352567296.000000,892029384.000000,2228224.000000,900.510000,63576969216.000000,132011507712.000000,114319.000000,29935.000000,911601.000000,938865.000000,3129858560.000000,89521949696.000000,455101.000000,6621992.000000 +239280152.000000,294674432.000000,892029384.000000,2260992.000000,942.170000,63586906112.000000,132011507712.000000,114319.000000,29935.000000,912015.000000,939414.000000,3129858560.000000,89562078208.000000,455262.000000,6622541.000000 +347575656.000000,376143872.000000,892029384.000000,2260992.000000,942.810000,63596789760.000000,132011507712.000000,114319.000000,29935.000000,912121.000000,939576.000000,3129858560.000000,89572891648.000000,455310.000000,6622685.000000 +348935136.000000,377700352.000000,892029384.000000,2260992.000000,942.870000,63596789760.000000,132011507712.000000,114319.000000,29935.000000,912159.000000,939619.000000,3129858560.000000,89573338112.000000,455313.000000,6622723.000000 +350310176.000000,379166720.000000,892029384.000000,2293760.000000,942.930000,63596789760.000000,132011507712.000000,114319.000000,29935.000000,912172.000000,939644.000000,3129858560.000000,89573493760.000000,455315.000000,6622736.000000 +401366528.000000,418070528.000000,892029384.000000,2293760.000000,981.380000,63609491456.000000,132011507712.000000,114319.000000,29935.000000,912456.000000,940011.000000,3129858560.000000,89596005376.000000,455422.000000,6623091.000000 +334867376.000000,361021440.000000,892029384.000000,2195456.000000,984.920000,63613825024.000000,132011507712.000000,114320.000000,29935.000000,912657.000000,940260.000000,3129862656.000000,89619520512.000000,455514.000000,6623432.000000 +336264608.000000,362430464.000000,892029384.000000,2195456.000000,984.970000,63613825024.000000,132011507712.000000,114320.000000,29935.000000,912736.000000,940324.000000,3129862656.000000,89620184064.000000,455523.000000,6623491.000000 +336786960.000000,363126784.000000,892029384.000000,2195456.000000,985.030000,63613825024.000000,132011507712.000000,114320.000000,29935.000000,912736.000000,940324.000000,3129862656.000000,89620184064.000000,455523.000000,6623491.000000 +351857352.000000,368885760.000000,892029384.000000,2326528.000000,1023.070000,63621074944.000000,132011507712.000000,114321.000000,29935.000000,913218.000000,940893.000000,3129866752.000000,89675226112.000000,455728.000000,6624431.000000 +391189520.000000,419880960.000000,892029384.000000,2293760.000000,1028.490000,63629893632.000000,132011507712.000000,114321.000000,29935.000000,913322.000000,941056.000000,3129866752.000000,89684982784.000000,455788.000000,6624587.000000 +392652096.000000,421306368.000000,892029384.000000,2293760.000000,1028.550000,63629893632.000000,132011507712.000000,114321.000000,29935.000000,913363.000000,941101.000000,3129866752.000000,89685429248.000000,455791.000000,6624622.000000 +394060712.000000,422887424.000000,892029384.000000,2293760.000000,1028.600000,63629893632.000000,132011507712.000000,114321.000000,29935.000000,913376.000000,941131.000000,3129866752.000000,89685605376.000000,455794.000000,6624635.000000 +351935208.000000,369205248.000000,892029384.000000,2260992.000000,1064.490000,63638695936.000000,132011507712.000000,114321.000000,29935.000000,913724.000000,941649.000000,3129866752.000000,89720781824.000000,455939.000000,6625138.000000 +293986560.000000,328458240.000000,892029384.000000,2195456.000000,1070.580000,63649800192.000000,132011507712.000000,114321.000000,29935.000000,913834.000000,941818.000000,3129866752.000000,89732848640.000000,455987.000000,6625288.000000 +295344848.000000,329850880.000000,892029384.000000,2195456.000000,1070.650000,63649800192.000000,132011507712.000000,114321.000000,29935.000000,913918.000000,941856.000000,3129866752.000000,89733426176.000000,455993.000000,6625356.000000 +295913760.000000,330416128.000000,892029384.000000,2195456.000000,1070.710000,63649792000.000000,132011507712.000000,114321.000000,29935.000000,913930.000000,941880.000000,3129866752.000000,89733573632.000000,455997.000000,6625367.000000 +240596848.000000,274628608.000000,892029384.000000,2293760.000000,1104.400000,63662354432.000000,132011507712.000000,114321.000000,29935.000000,914530.000000,943233.000000,3129866752.000000,89761709056.000000,456246.000000,6625944.000000 +248172880.000000,286908416.000000,892029384.000000,2260992.000000,1112.540000,63666212864.000000,132011507712.000000,114321.000000,29935.000000,915257.000000,943470.000000,3129866752.000000,89891703808.000000,456653.000000,6628579.000000 +249468200.000000,288161792.000000,892029384.000000,2260992.000000,1112.600000,63666212864.000000,132011507712.000000,114321.000000,29935.000000,915283.000000,943493.000000,3129866752.000000,89891970048.000000,456656.000000,6628604.000000 +249973640.000000,288710656.000000,892029384.000000,2260992.000000,1112.660000,63666212864.000000,132011507712.000000,114321.000000,29935.000000,915323.000000,943576.000000,3129866752.000000,89892473856.000000,456673.000000,6628650.000000 +302335280.000000,340049920.000000,892029384.000000,2326528.000000,1158.020000,63661408256.000000,132011507712.000000,114321.000000,29935.000000,915911.000000,944145.000000,3129866752.000000,89980664832.000000,456928.000000,6630695.000000 +429684360.000000,451485696.000000,892029384.000000,2326528.000000,1158.750000,63668527104.000000,132011507712.000000,114321.000000,29935.000000,916078.000000,944363.000000,3129866752.000000,89994157056.000000,457002.000000,6630879.000000 +209887464.000000,250134528.000000,892029384.000000,2228224.000000,1158.880000,63668523008.000000,132011507712.000000,114321.000000,29935.000000,916110.000000,944393.000000,3129866752.000000,89994513408.000000,457005.000000,6630913.000000 +211134312.000000,251265024.000000,892029384.000000,2228224.000000,1158.930000,63668523008.000000,132011507712.000000,114321.000000,29935.000000,916135.000000,944450.000000,3129866752.000000,89994849280.000000,457020.000000,6630949.000000 +257293280.000000,305676288.000000,892029384.000000,2392064.000000,1199.950000,63678472192.000000,132011507712.000000,114321.000000,29935.000000,916483.000000,944927.000000,3129866752.000000,90029890560.000000,457166.000000,6631455.000000 +256067288.000000,294313984.000000,892029384.000000,2293760.000000,1201.340000,63688945664.000000,132011507712.000000,114321.000000,29935.000000,916591.000000,945093.000000,3129866752.000000,90041318400.000000,457220.000000,6631607.000000 +257325048.000000,295747584.000000,892029384.000000,2293760.000000,1201.390000,63688945664.000000,132011507712.000000,114321.000000,29935.000000,916681.000000,945160.000000,3129866752.000000,90042117120.000000,457227.000000,6631686.000000 +258622120.000000,297304064.000000,892029384.000000,2326528.000000,1201.440000,63688945664.000000,132011507712.000000,114321.000000,29935.000000,916683.000000,945160.000000,3129866752.000000,90042125312.000000,457227.000000,6631688.000000 +299840152.000000,327041024.000000,892029384.000000,2326528.000000,1243.670000,63729782784.000000,132011507712.000000,114322.000000,29935.000000,917088.000000,945622.000000,3129870848.000000,90100435968.000000,457406.000000,6633057.000000 +326690152.000000,352878592.000000,892029384.000000,2260992.000000,1250.900000,63651233792.000000,132011507712.000000,114322.000000,29935.000000,917586.000000,945897.000000,3129870848.000000,90193492992.000000,457787.000000,6648071.000000 +327977568.000000,354525184.000000,892029384.000000,2260992.000000,1250.960000,63651233792.000000,132011507712.000000,114322.000000,29935.000000,917632.000000,945940.000000,3129870848.000000,90193857536.000000,457790.000000,6648115.000000 +328485952.000000,355270656.000000,892029384.000000,2293760.000000,1251.010000,63651233792.000000,132011507712.000000,114322.000000,29935.000000,917643.000000,945966.000000,3129870848.000000,90194009088.000000,457793.000000,6648125.000000 +255574816.000000,286892032.000000,892029384.000000,2326528.000000,1285.150000,63663865856.000000,132011507712.000000,114322.000000,29935.000000,917929.000000,946333.000000,3129870848.000000,90225728512.000000,457926.000000,6648603.000000 +278908352.000000,309059584.000000,892029384.000000,2260992.000000,1293.120000,63676129280.000000,132011507712.000000,114322.000000,29935.000000,918042.000000,946499.000000,3129870848.000000,90238942208.000000,457982.000000,6648775.000000 +280189448.000000,310575104.000000,892029384.000000,2260992.000000,1293.170000,63676129280.000000,132011507712.000000,114322.000000,29935.000000,918075.000000,946532.000000,3129870848.000000,90239261696.000000,457985.000000,6648811.000000 +281526944.000000,312008704.000000,892029384.000000,2260992.000000,1293.220000,63676129280.000000,132011507712.000000,114322.000000,29935.000000,918091.000000,946556.000000,3129870848.000000,90239425536.000000,457988.000000,6648829.000000 +321758296.000000,357138432.000000,892029384.000000,2326528.000000,1334.920000,63685742592.000000,132011507712.000000,114323.000000,29935.000000,918479.000000,947161.000000,3129874944.000000,90273950720.000000,458159.000000,6649357.000000 +404728504.000000,427229184.000000,892029384.000000,2260992.000000,1336.360000,63689998336.000000,132011507712.000000,114323.000000,29935.000000,918698.000000,947380.000000,3129874944.000000,90300517376.000000,458236.000000,6649703.000000 +406066824.000000,428736512.000000,892029384.000000,2260992.000000,1336.420000,63689998336.000000,132011507712.000000,114323.000000,29935.000000,918771.000000,947404.000000,3129874944.000000,90300996608.000000,458242.000000,6649754.000000 +407381896.000000,430374912.000000,892029384.000000,2293760.000000,1336.480000,63689998336.000000,132011507712.000000,114323.000000,29935.000000,918799.000000,947474.000000,3129874944.000000,90301398016.000000,458256.000000,6649792.000000 +248985056.000000,311910400.000000,892029384.000000,2359296.000000,1375.840000,63702720512.000000,132011507712.000000,114323.000000,29935.000000,919081.000000,947923.000000,3129874944.000000,90324880384.000000,458387.000000,6650139.000000 +367452424.000000,392192000.000000,892029384.000000,2260992.000000,1377.970000,63707570176.000000,132011507712.000000,114324.000000,29935.000000,919276.000000,948157.000000,3129879040.000000,90347056128.000000,458457.000000,6650434.000000 +368794632.000000,393863168.000000,892029384.000000,2260992.000000,1378.030000,63707570176.000000,132011507712.000000,114324.000000,29935.000000,919303.000000,948184.000000,3129879040.000000,90347453440.000000,458460.000000,6650460.000000 +369418240.000000,394444800.000000,892029384.000000,2260992.000000,1378.080000,63707570176.000000,132011507712.000000,114324.000000,29935.000000,919326.000000,948244.000000,3129879040.000000,90347793408.000000,458473.000000,6650495.000000 +273971304.000000,306610176.000000,892029384.000000,2326528.000000,1417.370000,63713411072.000000,132011507712.000000,114325.000000,29935.000000,919786.000000,948793.000000,3129883136.000000,90400738304.000000,458644.000000,6651183.000000 +360067328.000000,386875392.000000,892029384.000000,2293760.000000,1421.810000,63722237952.000000,132011507712.000000,114325.000000,29935.000000,919888.000000,948958.000000,3129883136.000000,90410511360.000000,458699.000000,6651319.000000 +361578960.000000,388513792.000000,892029384.000000,2293760.000000,1421.870000,63722237952.000000,132011507712.000000,114325.000000,29935.000000,919975.000000,948993.000000,3129883136.000000,90411031552.000000,458705.000000,6651386.000000 +363034648.000000,390062080.000000,892029384.000000,2293760.000000,1421.910000,63722237952.000000,132011507712.000000,114325.000000,29935.000000,919988.000000,949013.000000,3129883136.000000,90411166720.000000,458708.000000,6651399.000000 +395768520.000000,412950528.000000,892029384.000000,2326528.000000,1457.170000,63731904512.000000,132011507712.000000,114325.000000,29935.000000,920305.000000,949431.000000,3129883136.000000,90445265920.000000,458850.000000,6651877.000000 +321255112.000000,349708288.000000,892029384.000000,2260992.000000,1464.420000,63742767104.000000,132011507712.000000,114326.000000,29935.000000,920414.000000,949593.000000,3129887232.000000,90457062400.000000,458899.000000,6652017.000000 +322547288.000000,351518720.000000,892029384.000000,2260992.000000,1464.470000,63742767104.000000,132011507712.000000,114326.000000,29935.000000,920452.000000,949624.000000,3129887232.000000,90457345024.000000,458902.000000,6652053.000000 +323064648.000000,352239616.000000,892029384.000000,2260992.000000,1464.530000,63742767104.000000,132011507712.000000,114326.000000,29935.000000,920468.000000,949650.000000,3129887232.000000,90457517056.000000,458905.000000,6652069.000000 +324013144.000000,366125056.000000,892029384.000000,2326528.000000,1506.320000,63753170944.000000,132011507712.000000,114326.000000,29935.000000,920846.000000,950158.000000,3129887232.000000,90495532032.000000,459056.000000,6652606.000000 +436467688.000000,463945728.000000,892029384.000000,2326528.000000,1506.980000,63759286272.000000,132011507712.000000,114326.000000,29935.000000,920953.000000,950328.000000,3129887232.000000,90506857472.000000,459108.000000,6652752.000000 +437135216.000000,464617472.000000,892029384.000000,2326528.000000,1507.030000,63759286272.000000,132011507712.000000,114326.000000,29935.000000,920995.000000,950365.000000,3129887232.000000,90507242496.000000,459111.000000,6652791.000000 +438670808.000000,466272256.000000,892029384.000000,2326528.000000,1507.090000,63759282176.000000,132011507712.000000,114326.000000,29935.000000,921042.000000,950389.000000,3129887232.000000,90507545600.000000,459117.000000,6652827.000000 +234362352.000000,295239680.000000,892029384.000000,2260992.000000,1549.160000,63766687744.000000,132011507712.000000,114327.000000,29935.000000,922339.000000,951805.000000,3129891328.000000,90673024000.000000,459745.000000,6663341.000000 +338122280.000000,367788032.000000,892029384.000000,2260992.000000,1549.740000,63775334400.000000,132011507712.000000,114327.000000,29935.000000,922442.000000,951968.000000,3129891328.000000,90682612736.000000,459797.000000,6663477.000000 +339439464.000000,369344512.000000,892029384.000000,2260992.000000,1549.800000,63775334400.000000,132011507712.000000,114327.000000,29935.000000,922499.000000,952002.000000,3129891328.000000,90683042816.000000,459801.000000,6663520.000000 +340713832.000000,370966528.000000,892029384.000000,2260992.000000,1549.840000,63775338496.000000,132011507712.000000,114327.000000,29935.000000,922517.000000,952029.000000,3129891328.000000,90683227136.000000,459811.000000,6663544.000000 +396025560.000000,416022528.000000,892029384.000000,2326528.000000,1590.140000,63784239104.000000,132011507712.000000,114327.000000,29935.000000,922800.000000,952407.000000,3129891328.000000,90715843584.000000,459934.000000,6663998.000000 +308094120.000000,341278720.000000,892029384.000000,2228224.000000,1590.940000,63795916800.000000,132011507712.000000,114327.000000,29935.000000,922910.000000,952570.000000,3129891328.000000,90728451072.000000,459981.000000,6664163.000000 +309457096.000000,342745088.000000,892029384.000000,2228224.000000,1590.990000,63795916800.000000,132011507712.000000,114327.000000,29935.000000,922955.000000,952601.000000,3129891328.000000,90728762368.000000,459984.000000,6664195.000000 +310855800.000000,344244224.000000,892029384.000000,2260992.000000,1591.060000,63795916800.000000,132011507712.000000,114327.000000,29935.000000,922968.000000,952623.000000,3129891328.000000,90728905728.000000,459986.000000,6664206.000000 +372472568.000000,391905280.000000,892029384.000000,2293760.000000,1631.310000,63805771776.000000,132011507712.000000,114327.000000,29935.000000,923287.000000,953064.000000,3129891328.000000,90764184576.000000,460122.000000,6664716.000000 +434987728.000000,461324288.000000,892029384.000000,2260992.000000,1633.020000,63812075520.000000,132011507712.000000,114327.000000,29935.000000,923398.000000,953238.000000,3129891328.000000,90775735296.000000,460182.000000,6664876.000000 +436475552.000000,462733312.000000,892029384.000000,2260992.000000,1633.080000,63812075520.000000,132011507712.000000,114327.000000,29935.000000,923487.000000,953280.000000,3129891328.000000,90776366080.000000,460188.000000,6664943.000000 +437839616.000000,464396288.000000,892029384.000000,2293760.000000,1633.130000,63812075520.000000,132011507712.000000,114327.000000,29935.000000,923508.000000,953310.000000,3129891328.000000,90776574976.000000,460191.000000,6664964.000000 +302824216.000000,331333632.000000,892029384.000000,2293760.000000,1672.960000,63820083200.000000,132011507712.000000,114327.000000,29935.000000,924024.000000,953968.000000,3129891328.000000,90832346112.000000,460371.000000,6665869.000000 +428324424.000000,452198400.000000,892029384.000000,2326528.000000,1676.460000,63828606976.000000,132011507712.000000,114327.000000,29935.000000,924124.000000,954130.000000,3129891328.000000,90841803776.000000,460420.000000,6665997.000000 +429922648.000000,453746688.000000,892029384.000000,2326528.000000,1676.520000,63828606976.000000,132011507712.000000,114327.000000,29935.000000,924178.000000,954174.000000,3129891328.000000,90842242048.000000,460424.000000,6666041.000000 +431455768.000000,455213056.000000,892029384.000000,2326528.000000,1676.580000,63828606976.000000,132011507712.000000,114327.000000,29935.000000,924189.000000,954200.000000,3129891328.000000,90842393600.000000,460425.000000,6666051.000000 +410268600.000000,429359104.000000,892029384.000000,2326528.000000,1715.340000,63823212544.000000,132011507712.000000,114327.000000,29935.000000,924804.000000,954882.000000,3129891328.000000,90918894592.000000,460714.000000,6667936.000000 +378320944.000000,409804800.000000,892029384.000000,2326528.000000,1721.660000,63832317952.000000,132011507712.000000,114327.000000,29935.000000,924906.000000,955040.000000,3129891328.000000,90928917504.000000,460765.000000,6668073.000000 +379705464.000000,411254784.000000,892029384.000000,2326528.000000,1721.720000,63832317952.000000,132011507712.000000,114327.000000,29935.000000,924952.000000,955090.000000,3129891328.000000,90929359872.000000,460768.000000,6668113.000000 +380354824.000000,411803648.000000,892029384.000000,2326528.000000,1721.780000,63832317952.000000,132011507712.000000,114327.000000,29935.000000,924964.000000,955113.000000,3129891328.000000,90929503232.000000,460772.000000,6668126.000000 +431552576.000000,451264512.000000,892029384.000000,2326528.000000,1764.450000,63831511040.000000,132011507712.000000,114327.000000,29935.000000,925470.000000,955738.000000,3129891328.000000,90988424192.000000,460991.000000,6669715.000000 +366189440.000000,391520256.000000,892029384.000000,2228224.000000,1765.360000,63839870976.000000,132011507712.000000,114327.000000,29935.000000,925636.000000,955972.000000,3129891328.000000,91004967936.000000,461069.000000,6669952.000000 +367680600.000000,393011200.000000,892029384.000000,2228224.000000,1765.420000,63839870976.000000,132011507712.000000,114327.000000,29935.000000,925733.000000,956020.000000,3129891328.000000,91005725696.000000,461076.000000,6670027.000000 +369027928.000000,394485760.000000,892029384.000000,2260992.000000,1765.470000,63839870976.000000,132011507712.000000,114327.000000,29935.000000,925747.000000,956051.000000,3129891328.000000,91005910016.000000,461079.000000,6670040.000000 +333798608.000000,380764160.000000,892029384.000000,2293760.000000,1810.640000,63919005696.000000,132011507712.000000,114327.000000,29935.000000,925924.000000,956332.000000,3129891328.000000,91026258944.000000,461163.000000,6670305.000000 +242130464.000000,278732800.000000,892029384.000000,2228224.000000,1812.150000,63812001792.000000,132011507712.000000,114327.000000,29935.000000,926433.000000,956611.000000,3129891328.000000,91122723840.000000,461544.000000,6676148.000000 +243407136.000000,280125440.000000,892029384.000000,2228224.000000,1812.210000,63812001792.000000,132011507712.000000,114327.000000,29935.000000,926484.000000,956637.000000,3129891328.000000,91123096576.000000,461547.000000,6676190.000000 +243978768.000000,280608768.000000,892029384.000000,2228224.000000,1812.260000,63812001792.000000,132011507712.000000,114327.000000,29935.000000,926505.000000,956670.000000,3129891328.000000,91123317760.000000,461550.000000,6676209.000000 +371007152.000000,387579904.000000,892029384.000000,2293760.000000,1850.930000,63828652032.000000,132011507712.000000,114327.000000,29935.000000,926789.000000,957052.000000,3129891328.000000,91145636864.000000,461655.000000,6676573.000000 diff --git a/test/stress/istio/testdata/metrics/intermediate.csv b/test/stress/istio/testdata/metrics/intermediate.csv index 46304636d..973552fa0 100644 --- a/test/stress/istio/testdata/metrics/intermediate.csv +++ b/test/stress/istio/testdata/metrics/intermediate.csv @@ -1,9 +1,9 @@ -weighted_io,rate[5m],39.39,34.57,38.693333333333335,38.63333333333333,38.733333333333334,30.733333333333334,35.913333333333334,35.763333333333335,35.91,30.23,38.53,37.7,37.89,29.196666666666665,40.156666666666666,39.95666666666666,40.10666666666667,35.02,48.10333333333333,47.946666666666665,48.07666666666667,39.68333333333333,45.78333333333333,45.63666666666666,45.78,40.54,48.85,48.79,49.24333333333333,40.593333333333334,46.77,46.95666666666666,47.32666666666667,36.63666666666666,44.42666666666667,44.276666666666664,44.38333333333333,31.10333333333333,36.733333333333334,36.653333333333336,36.70666666666666,30.62,41.71333333333333,41.52333333333333,41.62,33.31333333333333,38.35,38,38.123333333333335,32.01,40.03666666666667,39.64,39.54666666666667,31.8,43.86,43.693333333333335,43.9,38.35666666666667,46.19,46.373333333333335,46.43333333333333,35.42,41.873333333333335,41.763333333333335,42.04333333333334,36.91,42.14333333333333,42.02333333333333,42.16,33.78333333333333,42.86666666666667,42.693333333333335,43.49333333333333,31.506666666666668,36.89666666666667,36.75666666666667,36.763333333333335,28.87,36.25,36.24666666666667,36.263333333333335,29.823333333333334,48.51,48.233333333333334,48.37,43.06333333333333,51.276666666666664,51.086666666666666,51.21,42.156666666666666,54.57333333333333,53.78333333333333,53.843333333333334,48.513333333333335,55.946666666666665,55.84,55.95,48.336666666666666,54.903333333333336,54.8,54.97,36.25,44.11333333333334,43.92,44.163333333333334,35.99,43.07666666666667,42.91,43.03666666666667,30.653333333333332,44.56333333333333,44.42666666666667,45.13,37.8,44.78,44.63,44.73,38.08,68.45333333333333,68.33333333333333,68.40666666666667,60.60333333333333,80.45666666666666,80.34333333333333,80.46333333333334,73.42666666666666,80.80666666666667,80.66333333333333,80.77666666666667,66.86333333333333,73.05333333333333,72.31666666666666,72.27333333333333,65.35 -io_time,rate[5m],12.573333333333334,10.933333333333334,12.48,12.48,12.466666666666667,10.306666666666667,11.96,11.88,11.866666666666667,8.813333333333333,11.186666666666667,11.053333333333333,11.08,9.28,11.12,11.093333333333334,11.12,9.893333333333333,12.16,12.133333333333333,12.16,10.053333333333333,12.053333333333333,12.013333333333334,12.013333333333334,10.293333333333333,12.28,12.36,12.44,9.893333333333333,13.066666666666666,13.16,13.24,11.466666666666667,13.386666666666667,13.36,13.333333333333334,10.88,12.786666666666667,12.733333333333333,12.746666666666666,10.733333333333333,13.28,13.226666666666667,13.2,11.213333333333333,12.906666666666666,12.813333333333333,12.893333333333333,9.8,11.613333333333333,11.52,11.493333333333334,9.613333333333333,11.573333333333334,11.546666666666667,11.586666666666666,9.666666666666666,11.693333333333333,11.746666666666666,11.773333333333333,9.253333333333334,10.786666666666667,10.8,10.866666666666667,9.093333333333334,10.96,10.906666666666666,10.88,8.96,12.413333333333334,12.4,12.506666666666666,10.586666666666666,12.133333333333333,12.093333333333334,12.12,10.093333333333334,12.186666666666667,12.146666666666667,12.186666666666667,10.693333333333333,12.733333333333333,12.693333333333333,12.72,10.88,12.866666666666667,12.853333333333333,12.866666666666667,9.44,11.346666666666666,11.253333333333334,11.28,9.773333333333333,11.493333333333334,11.453333333333333,11.493333333333334,9.36,11.373333333333333,11.28,11.266666666666667,9.226666666666667,11.253333333333334,11.186666666666667,11.226666666666667,9.266666666666667,11.28,11.226666666666667,11.226666666666667,9.32,12.56,12.48,12.546666666666667,10.853333333333333,12.906666666666666,12.826666666666666,12.813333333333333,10.813333333333333,13.053333333333333,13.066666666666666,13.066666666666666,11.066666666666666,13.506666666666666,13.533333333333333,13.546666666666667,11.52,13.573333333333334,13.56,13.6,10.36,12.08,12.013333333333334,12.026666666666667,10 -write_bytes,rate[5m],2118382.933333333,1811319.4666666666,2035165.8666666667,2034578.7733333334,2036039.68,1601085.44,1905322.6666666667,1903588.6933333334,1905377.28,1588893.0133333334,2037391.36,1996731.7333333334,1999298.56,1560057.1733333333,2051331.4133333333,2048628.0533333332,2050771.6266666667,1763519.1466666667,2339266.56,2337846.6133333333,2339321.1733333333,1884119.04,2224250.88,2222653.44,2224592.2133333334,1910087.68,2326091.0933333333,2329326.933333333,2333859.84,1862260.0533333332,2224319.1466666665,2228551.68,2265210.88,1776134.8266666667,2194732.3733333335,2193025.7066666665,2195101.013333333,1607352.32,1924164.2666666666,1923276.8,1924000.4266666668,1583240.5333333334,2160230.4,2158196.053333333,2159220.053333333,1743189.3333333333,2046457.1733333333,2042852.6933333334,2044354.56,1682623.1466666667,2131135.1466666665,2094298.4533333334,2093957.12,1675578.0266666666,2187769.1733333333,2185448.1066666665,2187714.56,1871298.56,2270044.16,2277894.8266666667,2278386.3466666667,1701806.08,2071169.7066666668,2070773.76,2073436.16,1764939.0933333333,2070391.4666666666,2068889.6,2071183.36,1615831.04,2090557.44,2088823.4666666666,2129715.2,1617715.2,1914975.5733333332,1913350.8266666667,1913992.5333333334,1514482.3466666667,1933407.5733333332,1933544.1066666667,1934062.9333333333,1564617.3866666667,2117399.8933333335,2114833.066666667,2116635.3066666666,1810459.3066666666,2229971.6266666665,2227527.68,2228961.28,1754426.0266666666,2265729.7066666665,2227568.64,2228619.946666667,1931769.1733333333,2311768.7466666666,2310594.56,2312492.3733333335,1885033.8133333332,2259476.48,2258479.7866666666,2261046.6133333333,1707458.56,2097711.7866666666,2095568.2133333334,2097943.8933333335,1678595.4133333333,2063592.1066666667,2062008.32,2063441.92,1552192.8533333333,2055727.7866666666,2054184.96,2094039.04,1714653.8666666667,2095199.5733333332,2093056,2094120.96,1718640.64,2368389.12,2366300.16,2367460.6933333334,1977548.8,2694758.4,2705476.2666666666,2707019.0933333333,2322377.3866666667,2720959.1466666665,2719416.32,2720918.1866666665,2214584.32,2572916.053333333,2533744.64,2533976.7466666666,2153758.72 -read_bytes,rate[5m],0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 -merged_write_count,rate[5m],24.006666666666668,21.19,23.033333333333335,22.973333333333333,23.156666666666666,19.013333333333332,21.99,21.813333333333333,21.98,15.903333333333334,20.466666666666665,20.14,20.323333333333334,17.14,19.856666666666666,19.643333333333334,19.9,17.533333333333335,21.49,21.37,21.516666666666666,17.526666666666667,21.30666666666667,21.14,21.346666666666668,18.09,21.8,21.946666666666665,22.226666666666667,17.316666666666666,24.39,24.643333333333334,24.683333333333334,22.036666666666665,25.326666666666668,25.166666666666668,25.333333333333332,20.936666666666667,24.59,24.526666666666667,24.58,20.723333333333333,25.256666666666668,25.063333333333333,25.22,21.496666666666666,24.16,24.013333333333332,24.12,17.05666666666667,20.296666666666667,20.246666666666666,20.37,17.096666666666668,20.80666666666667,20.636666666666667,20.85333333333333,17.166666666666668,21.43,21.696666666666665,21.716666666666665,17.2,20.466666666666665,20.43,20.623333333333335,17.61,20.876666666666665,20.746666666666666,20.996666666666666,17.283333333333335,24.553333333333335,24.366666666666667,24.726666666666667,21.006666666666668,23.496666666666666,23.356666666666666,23.45,19.063333333333333,22.98,23.043333333333333,23.086666666666666,19.803333333333335,23.223333333333333,23.053333333333335,23.24,19.866666666666667,24.043333333333333,23.8,23.966666666666665,16.703333333333333,19.533333333333335,19.30666666666667,19.406666666666666,16.956666666666667,19.82,19.73,19.883333333333333,15.653333333333334,19.446666666666665,19.376666666666665,19.563333333333333,16.03,20.21,19.993333333333332,20.16,15.996666666666666,20.18,20,20.166666666666668,17.33,23.683333333333334,23.536666666666665,23.833333333333332,20.926666666666666,24.753333333333334,24.586666666666666,24.683333333333334,20.736666666666668,24.156666666666666,24.1,24.24,20.08,23.92,23.96,24.136666666666667,19.996666666666666,23.916666666666668,23.763333333333332,23.93,17.413333333333334,20.72,20.486666666666668,20.58,16.796666666666667 -write_count,rate[5m],19.513333333333332,16.876666666666665,18.316666666666666,18.266666666666666,18.363333333333333,14.946666666666667,17.436666666666667,17.276666666666667,17.406666666666666,14.343333333333334,18.11,17.26,17.413333333333334,14.186666666666667,17.283333333333335,17.083333333333332,17.236666666666668,15.096666666666666,19.163333333333334,19,19.12,15.61,18.586666666666666,18.44,18.546666666666667,15.98,19.286666666666665,19.323333333333334,19.926666666666666,15.906666666666666,19.523333333333333,19.92,20.07,17.323333333333334,20.453333333333333,20.31,20.42,16.08,18.846666666666668,18.77,18.82,15.84,20.136666666666667,19.973333333333333,20.066666666666666,16.766666666666666,19.07,18.526666666666667,18.676666666666666,15.13,18.083333333333332,17.9,17.713333333333335,14.62,18.273333333333333,18.106666666666666,18.253333333333334,15.63,18.97,19.093333333333334,19.133333333333333,14.92,17.946666666666665,17.816666666666666,18.133333333333333,15.65,18.226666666666667,18.08,18.19333333333333,14.716666666666667,18.913333333333334,18.753333333333334,19.576666666666668,15.976666666666667,18.29,18.206666666666667,18.153333333333332,14.766666666666667,18.13,18.136666666666667,18.153333333333332,15.143333333333333,18.736666666666668,18.42,18.563333333333333,15.95,19.613333333333333,19.44,19.563333333333333,15.406666666666666,18.79,18,18.1,15.846666666666666,18.643333333333334,18.546666666666667,18.67,15.136666666666667,18.273333333333333,18.166666666666668,18.34,14.733333333333333,18.14666666666667,17.95,18.226666666666667,14.61,18.083333333333332,17.916666666666668,18.066666666666666,14.723333333333333,18.56,18.39,18.976666666666667,16.35333333333333,19.703333333333333,19.533333333333335,19.653333333333332,16.44,20.723333333333333,20.536666666666665,20.623333333333335,17.273333333333333,21.9,21.713333333333335,21.846666666666668,18.433333333333334,21.75,21.56,21.69,17.833333333333332,20.706666666666667,20.113333333333333,20.006666666666668,16.713333333333335 -merged_read_count,rate[5m],0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 -read_count,rate[5m],0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 -process_cpu_seconds_total,rate[5m],1.149333333333333,0.9892333333333333,1.2094000000000003,1.2094000000000003,1.1938666666666666,0.9931666666666666,1.201833333333333,1.2018666666666666,1.1878000000000002,1.0065,1.2155999999999998,1.2156333333333333,1.1708333333333332,1.0128333333333337,1.2228666666666665,1.2228333333333334,1.2073,1.0358333333333336,1.2448000000000001,1.2447666666666668,1.2188666666666668,1.0423666666666667,1.2391,1.2390999999999999,1.2092333333333334,1.0161333333333333,1.2443,1.2444666666666664,1.2057333333333333,1.0175,1.2348,1.2349333333333334,1.1947333333333332,1.0049000000000001,1.2295333333333334,1.2295333333333334,1.183333333333333,0.999533333333333,1.2142333333333333,1.2142333333333328,1.170566666666667,1.008233333333333,1.2313333333333332,1.2314000000000003,1.2152666666666667,1.0212999999999999,1.2252333333333332,1.2253,1.2027333333333334,1.0241000000000005,1.2316333333333334,1.231633333333333,1.2114333333333336,1.0334999999999999,1.2389000000000003,1.2389000000000003,1.2134,1.0148333333333335,1.2406333333333335,1.2409999999999999,1.2066000000000001,1.0105666666666668,1.2260999999999997,1.2260999999999997,1.191933333333334,1.0211666666666672,1.2261999999999997,1.226233333333333,1.1875666666666667,1.0218999999999998,1.2302000000000006,1.2301333333333333,1.1834666666666664,1.0151,1.2191,1.2191,1.2029666666666663,1.0220000000000005,1.227833333333333,1.2278000000000004,1.2007666666666668,1.0342333333333333,1.2448333333333335,1.2447666666666661,1.2114333333333336,1.0559999999999998,1.2516000000000007,1.2515666666666667,1.209633333333333,1.022566666666666,1.254333333333334,1.2543999999999997,1.2173666666666667,1.0376000000000007,1.2558666666666667,1.2559,1.2110333333333323,1.0367,1.2546000000000004,1.2545666666666664,1.205533333333333,1.028433333333334,1.2388333333333337,1.2388333333333337,1.1892,1.0357666666666667,1.238166666666666,1.2381333333333335,1.2169000000000005,1.0335999999999999,1.2364666666666668,1.2364333333333328,1.2121999999999995,1.0360999999999998,1.2393666666666665,1.2394000000000005,1.2075666666666667,1.0378999999999996,1.2558000000000007,1.2557333333333343,1.222233333333332,1.0316999999999992,1.2995333333333323,1.3002666666666665,1.2580999999999996,1.1058999999999999,1.3021333333333345,1.3020666666666663,1.253733333333333,1.0739999999999994,1.2974666666666674,1.2975333333333325,1.255333333333333,1.082200000000001 +weighted_io,rate[5m],50.95333333333333,37.95,38.52333333333333,38.85,38.806666666666665,14.096666666666666,12.63,14.243333333333334,14.293333333333333,12.933333333333334,13.37,15.096666666666666,15.063333333333333,13.836666666666666,14.723333333333333,81.9,82.28666666666666,80.28666666666666,79.23,80.82333333333334,80.89333333333333,79.78,85.64666666666666,85.94666666666667,85.97333333333333,84.86333333333333,122.10666666666667,122.52666666666667,122.67,121.71,120.55666666666667,139.88,139.93666666666667,138.57333333333332,72.75333333333333,73.46,73.52333333333333,72.46333333333334,72.61,73.00333333333333,73.00333333333333,66.61333333333333,67.31666666666666,68.25666666666666,68.20333333333333,29.513333333333332,31.43,31.723333333333333,31.883333333333333,31.01,13.416666666666666,13.773333333333333,13.816666666666666,12.263333333333334,12.26,13.166666666666666,13.293333333333333,11.363333333333333,13.97,14.413333333333334,14.46,13.39,13.993333333333334,14.493333333333334,14.493333333333334,11.956666666666667,13.433333333333334,21.993333333333332,22.04,20.363333333333333,26.7,27.186666666666667,27.256666666666668,26.19333333333333,26.743333333333332,27.053333333333335,27.316666666666666,24.19,28.233333333333334,78.16333333333333,78.26666666666667,76.62333333333333,77.71666666666667,78.06333333333333,78.14666666666666,76.28333333333333,69.26,70.33,70.34666666666666,63.656666666666666,64.2,65.07,65.03666666666666,63.46666666666667,65.25333333333333,65.44333333333333,65.66,61.14,12.686666666666667,13.006666666666666,13.093333333333334,11.553333333333333,12.77,13.136666666666667,13.206666666666667,11.566666666666666,45.46,45.74333333333333,45.76,44.68333333333333,45.21333333333333,45.67666666666667,45.666666666666664,43.41,44.656666666666666,44.96666666666667,45.14666666666667,43.623333333333335,46.17333333333333,46.48,46.57333333333333,44.81666666666667,50.61333333333334,50.94,50.95333333333333,15.95,20.793333333333333,21.44,21.61,20.14,20.473333333333333,39.843333333333334,39.946666666666665,38.31,38.99 +io_time,rate[5m],6.8933333333333335,4.973333333333334,5.263333333333334,5.3966666666666665,5.3533333333333335,3.3466666666666667,3.3966666666666665,3.7533333333333334,3.77,3.3966666666666665,3.48,3.8766666666666665,3.8633333333333333,3.4066666666666667,3.783333333333333,5.483333333333333,5.513333333333334,4.973333333333334,4.903333333333333,5.27,5.27,4.81,5.573333333333333,5.733333333333333,5.723333333333334,5.306666666666667,7.166666666666667,7.326666666666667,7.34,7.083333333333333,6.886666666666667,8.16,8.156666666666666,7.633333333333334,6.406666666666666,6.6466666666666665,6.6466666666666665,6.333333333333333,6.596666666666667,6.773333333333333,6.77,5.8533333333333335,6.116666666666666,6.333333333333333,6.323333333333333,4.096666666666667,4.586666666666667,4.736666666666666,4.746666666666667,4.523333333333333,3.78,3.93,3.92,3.433333333333333,3.51,3.8033333333333332,3.82,3.1733333333333333,3.67,3.8633333333333333,3.86,3.4433333333333334,3.7,3.86,3.8466666666666667,3.18,3.8466666666666667,5.183333333333334,5.183333333333334,4.703333333333333,5.3933333333333335,5.63,5.633333333333334,5.326666666666667,5.506666666666667,5.656666666666666,5.68,4.996666666666667,5.3933333333333335,6.653333333333333,6.653333333333333,6.18,6.463333333333333,6.63,6.626666666666667,5.806666666666667,5.02,5.266666666666667,5.23,4.426666666666667,4.616666666666666,4.84,4.8,4.3566666666666665,4.746666666666667,4.906666666666666,4.926666666666667,4.34,3.5433333333333334,3.6966666666666668,3.6966666666666668,3.263333333333333,3.58,3.743333333333333,3.743333333333333,3.1933333333333334,5.03,5.183333333333334,5.15,4.746666666666667,4.923333333333333,5.07,5.036666666666667,4.473333333333334,4.743333333333333,4.923333333333333,4.933333333333334,4.47,4.906666666666666,5.06,5.0633333333333335,4.5633333333333335,5.3533333333333335,5.513333333333334,5.503333333333333,3.4233333333333333,3.98,4.226666666666667,4.216666666666667,3.816666666666667,3.94,5.2,5.203333333333333,4.76,4.91 +write_bytes,rate[5m],1633498.4533333334,1243627.52,1277378.56,1317191.68,1315894.6133333333,838806.1866666666,743041.7066666667,856910.5066666667,857825.28,783988.0533333333,808099.84,943377.0666666667,941970.7733333333,856855.8933333333,907223.04,1296465.92,1299565.2266666666,1154976.4266666668,1089631.5733333332,1208019.6266666667,1208934.4,1130291.2,1392954.0266666666,1419687.2533333334,1420861.44,1345208.32,1806813.8666666667,1841780.0533333332,1844032.8533333333,1777978.0266666666,1705465.1733333333,2026960.2133333334,2027438.08,1935209.8133333332,1656299.52,1739202.56,1740008.1066666667,1667631.7866666666,1667235.84,1708086.6133333333,1708318.72,1403589.9733333334,1451417.6,1526046.72,1525268.48,951118.5066666667,1100062.72,1129294.5066666666,1130291.2,1065314.9866666666,876489.3866666667,911291.7333333333,912029.0133333333,800235.52,788493.6533333333,864706.56,866167.4666666667,746536.96,887507.6266666667,919033.1733333333,919838.72,844868.2666666667,885418.6666666666,925641.3866666667,925341.0133333333,739328,801559.8933333333,1233005.2266666666,1233401.1733333333,1101318.8266666667,1359243.9466666668,1402729.8133333332,1403398.8266666667,1329479.68,1367900.16,1403781.12,1406443.52,1222997.3333333333,1384843.9466666668,1693545.8133333332,1694173.8666666667,1577424.2133333334,1642932.9066666667,1685053.44,1685626.88,1592388.2666666666,1274156.3733333333,1361824.4266666668,1361742.5066666666,1069110.6133333333,1102411.0933333333,1175142.4,1175347.2,1059676.16,1198066.3466666667,1227980.8,1229687.4666666666,1035769.1733333333,839243.0933333334,877349.5466666666,877786.4533333334,772628.48,855299.4133333333,891985.92,892723.2,778649.6,1241688.7466666666,1272053.76,1272149.3333333333,1194489.1733333333,1229291.52,1269992.1066666667,1269896.5333333334,1093891.4133333333,1178910.72,1215679.1466666667,1217331.2,1104363.52,1250945.7066666668,1281529.1733333333,1282416.64,1156205.2266666666,1373457.0666666667,1405583.36,1406047.5733333332,854930.7733333333,1019371.52,1073083.7333333334,1074995.2,966888.1066666667,992692.9066666666,1313204.9066666667,1313969.4933333334,1197110.6133333333,1233005.2266666666 +read_bytes,rate[5m],73564.16,382.29333333333335,382.29333333333335,245.76,245.76,204.8,204.8,204.8,204.8,204.8,204.8,204.8,204.8,0,505.17333333333335,983.04,983.04,983.04,983.04,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,491.52,259.41333333333336,259.41333333333336,259.41333333333336,259.41333333333336,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,0,0,13.653333333333334,13.653333333333334,13.653333333333334,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,13.653333333333334,13.653333333333334,13.653333333333334,0,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,40.96,40.96,40.96,54.61333333333334,54.61333333333334,54.61333333333334,40.96,40.96,54.61333333333334,54.61333333333334,54.61333333333334,54.61333333333334,54.61333333333334,54.61333333333334,40.96,54.61333333333334,54.61333333333334,54.61333333333334,54.61333333333334,40.96,40.96,40.96,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,0,0,0,0,0,0,0,0,0,0 +merged_write_count,rate[5m],19.71,14.866666666666667,16.09,16.483333333333334,16.173333333333332,11.713333333333333,12.096666666666666,13.003333333333334,13.043333333333333,11.86,12.24,13.14,12.746666666666666,11.333333333333334,12.29,12.933333333333334,12.966666666666667,11.486666666666666,11.953333333333333,12.973333333333333,12.993333333333334,11.263333333333334,13.033333333333333,13.496666666666666,13.583333333333334,12.15,16.296666666666667,16.686666666666667,16.883333333333333,16.033333333333335,15.93,16.703333333333333,16.696666666666665,15.306666666666667,16.10333333333333,16.793333333333333,16.733333333333334,15.383333333333333,16.303333333333335,16.733333333333334,16.753333333333334,14.516666666666667,15.356666666666667,15.973333333333333,15.846666666666666,10.82,12.05,12.386666666666667,12.37,11.543333333333333,12.506666666666666,12.94,12.973333333333333,11.516666666666667,11.916666666666666,12.623333333333333,12.723333333333333,10.656666666666666,11.993333333333334,12.41,12.456666666666667,11.156666666666666,12.056666666666667,12.62,12.53,10.793333333333333,14.766666666666667,15.43,15.426666666666666,13.873333333333333,15.23,15.813333333333333,15.83,14.796666666666667,15.556666666666667,15.896666666666667,16.12,14.223333333333333,15.22,15.986666666666666,16.03,14.39,15.05,15.476666666666667,15.506666666666666,11.076666666666666,12.303333333333333,12.956666666666667,12.76,11.096666666666666,11.866666666666667,12.546666666666667,12.446666666666667,11.056666666666667,12.333333333333334,12.66,12.776666666666667,11.303333333333333,11.78,12.176666666666666,12.193333333333333,11.056666666666667,12.196666666666667,12.653333333333334,12.696666666666667,10.76,14.75,15.213333333333333,15.093333333333334,13.686666666666667,14.166666666666666,14.62,14.523333333333333,12.766666666666667,13.686666666666667,14.15,14.223333333333333,12.93,14.583333333333334,15.02,15.08,13.473333333333333,15.18,15.583333333333334,15.67,11.026666666666667,12.566666666666666,13.233333333333333,13.303333333333333,12.146666666666667,12.54,13.366666666666667,13.38,12.02,12.713333333333333 +write_count,rate[5m],19.15,13.23,13.97,13.966666666666667,13.98,9.996666666666666,9.686666666666667,10.64,10.756666666666666,9.496666666666666,9.75,10.33,10.326666666666666,9.366666666666667,10.006666666666666,11.386666666666667,11.936666666666667,10.796666666666667,10.65,11.38,11.436666666666667,10.363333333333333,12.3,12.48,12.533333333333333,11.646666666666667,14.91,15.193333333333333,15.396666666666667,14.836666666666666,14.376666666666667,15.916666666666666,15.986666666666666,15.04,14.416666666666666,14.516666666666667,14.653333333333334,13.823333333333334,14.303333333333333,14.55,14.576666666666666,12.33,12.943333333333333,13.483333333333333,13.42,9.436666666666667,10.413333333333334,10.54,10.75,10.2,9.92,10.13,10.183333333333334,9.136666666666667,9.343333333333334,9.726666666666667,9.92,8.453333333333333,9.686666666666667,9.926666666666666,10,9.11,9.58,9.946666666666667,9.97,8.426666666666666,10.083333333333334,12.226666666666667,12.273333333333333,11.026666666666667,12.633333333333333,13.063333333333333,13.126666666666667,12.263333333333334,12.753333333333334,12.85,13.15,11.55,12.553333333333333,14.076666666666666,14.186666666666667,13.063333333333333,13.65,13.746666666666666,13.816666666666666,11.87,10.74,11.383333333333333,11.493333333333334,9.626666666666667,10.01,10.553333333333333,10.56,9.476666666666667,10.65,10.69,10.973333333333333,9.666666666666666,9.063333333333333,9.273333333333333,9.363333333333333,8.463333333333333,9.346666666666666,9.593333333333334,9.68,8.543333333333333,12.136666666666667,12.236666666666666,12.333333333333334,11.453333333333333,11.746666666666666,12.023333333333333,12.096666666666666,10.606666666666667,11.33,11.41,11.663333333333334,10.676666666666666,12.033333333333333,12.24,12.366666666666667,11.143333333333333,12.836666666666666,13.036666666666667,13.033333333333333,8.75,10.093333333333334,10.456666666666667,10.72,9.823333333333334,10.046666666666667,11.593333333333334,11.72,10.726666666666667,11.303333333333333 +merged_read_count,rate[5m],0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +read_count,rate[5m],0.61,0.05,0.05,0.02666666666666667,0.02666666666666667,0.023333333333333334,0.023333333333333334,0.023333333333333334,0.023333333333333334,0.023333333333333334,0.023333333333333334,0.023333333333333334,0.023333333333333334,0,0.03,0.06,0.06,0.06,0.06,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.03333333333333333,0.03333333333333333,0.03333333333333333,0.03333333333333333,0.03333333333333333,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0,0,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.01,0.01,0.01,0.013333333333333334,0.013333333333333334,0.013333333333333334,0.01,0.01,0.013333333333333334,0.013333333333333334,0.013333333333333334,0.013333333333333334,0.013333333333333334,0.013333333333333334,0.01,0.013333333333333334,0.013333333333333334,0.013333333333333334,0.013333333333333334,0.01,0.01,0.01,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0,0,0,0,0,0,0,0,0,0 +process_cpu_seconds_total,rate[5m],0.7889666666666667,0.6147333333333332,0.7205000000000001,0.7329999999999999,0.733,0.6234666666666666,0.7028666666666668,0.7327333333333333,0.7327666666666666,0.5991000000000001,0.6992666666666666,0.7377333333333334,0.7377333333333334,0.6163333333333334,0.7236666666666666,0.7401333333333332,0.7401000000000001,0.621,0.6988333333333333,0.7290999999999999,0.7290666666666668,0.5989666666666664,0.7423333333333335,0.7440333333333332,0.7440333333333334,0.6263333333333333,0.7319000000000002,0.7480333333333334,0.7480333333333332,0.6394000000000002,0.7208,0.7589333333333333,0.7589333333333331,0.6310000000000002,0.7525666666666666,0.7612666666666666,0.7612666666666666,0.647566666666667,0.7412666666666666,0.7629666666666666,0.7629333333333334,0.6071,0.7163666666666669,0.745,0.7449666666666667,0.6094999999999997,0.7358666666666667,0.7379,0.7379,0.6179999999999999,0.7185666666666665,0.7204999999999999,0.7205333333333332,0.5825333333333333,0.7018333333333332,0.7134333333333333,0.7134333333333333,0.5895,0.6944333333333335,0.7123333333333335,0.7123333333333332,0.6013666666666662,0.6921999999999999,0.7123333333333332,0.7124000000000005,0.5699000000000001,0.6800000000000004,0.7069333333333331,0.7069666666666664,0.5683000000000004,0.7173666666666668,0.7196,0.7198333333333339,0.5918333333333335,0.716766666666667,0.721233333333333,0.7212000000000004,0.5945666666666667,0.7172666666666668,0.7411666666666671,0.7412000000000004,0.6217333333333332,0.7152333333333338,0.741566666666666,0.7415333333333335,0.6293999999999997,0.7412666666666671,0.7458666666666667,0.7458666666666667,0.5948666666666668,0.723633333333333,0.7302999999999997,0.7303333333333331,0.5937666666666662,0.7201,0.7347333333333328,0.7347666666666661,0.5941333333333334,0.6875666666666667,0.7115333333333335,0.7115333333333335,0.597933333333333,0.7106666666666668,0.7126999999999998,0.7126999999999998,0.5738999999999995,0.7093333333333339,0.7110666666666664,0.7110666666666664,0.58,0.7072333333333336,0.7097000000000002,0.7097000000000002,0.5789666666666669,0.6983333333333334,0.7038333333333336,0.7038999999999995,0.5865333333333335,0.6951333333333332,0.7066333333333333,0.7066333333333333,0.5675333333333333,0.694533333333333,0.7154333333333337,0.7154333333333337,0.5753999999999997,0.7157000000000001,0.7185333333333331,0.7186000000000005,0.5844333333333331,0.7323333333333335,0.7372000000000003,0.737166666666667,0.6031666666666669,0.7263666666666669 diff --git a/test/stress/istio/testdata/metrics/result.csv b/test/stress/istio/testdata/metrics/result.csv index 0c459e727..0a64f2904 100644 --- a/test/stress/istio/testdata/metrics/result.csv +++ b/test/stress/istio/testdata/metrics/result.csv @@ -1,16 +1,16 @@ Metric Name, Min, Max, Mean, Median, P90, P95, P98, P99 -go_memstats_alloc_bytes, 194574488.000000, 1219366472.000000, 636633944.000000, 638610360.000000, 903115996.000000, 1009815720.000000, 1123834736.000000, 1129067512.000000 -go_memstats_heap_inuse_bytes, 270991360.000000, 1269211136.000000, 752114105.725490, 736362496.000000, 1068589056.000000, 1184641024.000000, 1226076160.000000, 1239941120.000000 -go_memstats_sys_bytes, 677992856.000000, 1834329256.000000, 1277992721.307189, 1237026328.000000, 1722618512.000000, 1816863912.000000, 1825809576.000000, 1834329256.000000 -go_memstats_stack_inuse_bytes, 3866624.000000, 21168128.000000, 12528512.836601, 13107200.000000, 18759680.000000, 19988480.000000, 20447232.000000, 20725760.000000 -process_cpu_seconds_total, 0.989233, 1.302133, 1.177522, 1.215433, 1.254483, 1.257000, 1.299900, 1.301167 -disk_used, 314598531072.000000, 315101392896.000000, 314877542801.568604, 314868518912.000000, 315058540544.000000, 315079880704.000000, 315091447808.000000, 315101392896.000000 -disk_total, 532622512128.000000, 532622512128.000000, 532622512128.000000, 532622512128.000000, 532622512128.000000, 532622512128.000000, 532622512128.000000, 532622512128.000000 -read_count, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000 +go_memstats_alloc_bytes, 208037424.000000, 438670808.000000, 336485138.909091, 337454620.000000, 415208472.000000, 431504172.000000, 436471620.000000, 437487416.000000 +go_memstats_heap_inuse_bytes, 242704384.000000, 466272256.000000, 365532412.675325, 367603712.000000, 440102912.000000, 452972544.000000, 463339520.000000, 464506880.000000 +go_memstats_sys_bytes, 891505096.000000, 892029384.000000, 891923845.506493, 892029384.000000, 892029384.000000, 892029384.000000, 892029384.000000, 892029384.000000 +go_memstats_stack_inuse_bytes, 2162688.000000, 2392064.000000, 2280354.909091, 2293760.000000, 2326528.000000, 2359296.000000, 2359296.000000, 2375680.000000 +process_cpu_seconds_total, 0.567533, 0.788967, 0.693620, 0.713433, 0.744500, 0.755750, 0.762100, 0.762950 +disk_used, 63312080896.000000, 63919005696.000000, 63615498718.753250, 63617449984.000000, 63812075520.000000, 63830081536.000000, 63836094464.000000, 63839870976.000000 +disk_total, 132011507712.000000, 132011507712.000000, 132011507712.000000, 132011507712.000000, 132011507712.000000, 132011507712.000000, 132011507712.000000, 132011507712.000000 +read_count, 0.000000, 0.610000, 0.023926, 0.010000, 0.063333, 0.063333, 0.063333, 0.063333 merged_read_count, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000 -write_count, 14.186667, 21.900000, 18.114826, 18.260000, 20.125000, 20.715000, 21.731667, 21.798333 -merged_write_count, 15.653333, 25.333333, 21.324801, 21.073333, 24.566667, 24.740000, 25.238333, 25.291667 -read_bytes, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000 -write_bytes, 1514482.346667, 2720959.146667, 2074918.769353, 2089690.453333, 2338556.586667, 2553446.400000, 2713217.706667, 2720167.253333 -io_time, 8.813333, 13.600000, 11.646070, 11.653333, 13.113333, 13.373333, 13.553333, 13.566667 -weighted_io, 28.870000, 80.806667, 45.567090, 42.973333, 66.106667, 73.240000, 80.563333, 80.720000 +write_count, 8.426667, 19.150000, 11.598420, 11.383333, 14.340000, 14.873333, 15.656667, 15.951667 +merged_write_count, 10.656667, 19.710000, 13.701605, 13.043333, 16.111667, 16.700000, 16.773333, 16.838333 +read_bytes, 0.000000, 73564.160000, 756.596938, 40.960000, 996.693333, 996.693333, 996.693333, 996.693333 +write_bytes, 739328.000000, 2027438.080000, 1224245.045728, 1222997.333333, 1689586.346667, 1758993.066667, 1889621.333333, 1981085.013333 +io_time, 3.173333, 8.160000, 4.977457, 4.926667, 6.646667, 6.988333, 7.486667, 7.895000 +weighted_io, 11.363333, 139.936667, 46.949284, 43.623333, 81.396667, 121.133333, 130.621667, 139.226667 From 074f333ac9b4f8253762d82327068bf81742d190 Mon Sep 17 00:00:00 2001 From: Gao Hongtao Date: Fri, 15 Nov 2024 06:40:00 +0000 Subject: [PATCH 6/6] Update documents Signed-off-by: Gao Hongtao --- api/validate/validate.go | 3 +++ docs/concept/data-model.md | 20 ++++++++++++++++++++ 2 files changed, 23 insertions(+) diff --git a/api/validate/validate.go b/api/validate/validate.go index 1c4481aa2..9c404762a 100644 --- a/api/validate/validate.go +++ b/api/validate/validate.go @@ -132,6 +132,9 @@ func Measure(measure *databasev1.Measure) error { if len(measure.TagFamilies) == 0 { return errors.New("measure tag families is empty") } + if measure.IndexMode && len(measure.Fields) > 0 { + return errors.New("index mode is enabled, but fields are not empty") + } return tagFamily(measure.TagFamilies) } diff --git a/docs/concept/data-model.md b/docs/concept/data-model.md index 06726eb3e..e81c0640b 100644 --- a/docs/concept/data-model.md +++ b/docs/concept/data-model.md @@ -76,6 +76,7 @@ fields: entity: tag_names: - entity_id +index_mode: false interval: 1m ``` @@ -115,6 +116,25 @@ functions to them. Another option named `interval` plays a critical role in encoding. It indicates the time range between two adjacent data points in a time series and implies that all data points belonging to the same time series are distributed based on a fixed interval. A better practice for the naming measure is to append the interval literal to the tail, for example, `service_cpm_minute`. It's a parameter of `GORILLA` encoding method. +`index_mode` is a flag to enable the series index as the storage engine. All the tags will be stored in the inverted index and no field is allowed in the measure. This mode is suitable for the non-time series data model but needs TTL to be set. In this mode, the tags defined in the `entity` is the unique key of the data point. `timestamp` and `version` are the common tags in the inverted index. + +There is an example of a measure with the index mode enabled: + +```yaml +metadata: + name: service_traffic + group: sw_metric +tag_families: +- name: default + tags: + - name: id + type: TAG_TYPE_STRING + - name: service_name + type: TAG_TYPE_STRING +index_mode: true +entity: ["id"] +``` + [Measure Registration Operations](../api-reference.md#measureregistryservice) #### TopNAggregation